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 ae...@apache.org on 2016/03/18 19:25:01 UTC

[01/46] hadoop git commit: HADOOP-12672. RPC timeout should not override IPC ping interval (iwasakims)

Repository: hadoop
Updated Branches:
  refs/heads/HDFS-7240 4143f0362 -> 37e3a36a3


HADOOP-12672. RPC timeout should not override IPC ping interval (iwasakims)


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

Branch: refs/heads/HDFS-7240
Commit: 682adc6ba9db3bed94fd4ea3d83761db6abfe695
Parents: e01c6ea
Author: Masatake Iwasaki <iw...@apache.org>
Authored: Fri Mar 11 15:03:40 2016 +0900
Committer: Masatake Iwasaki <iw...@apache.org>
Committed: Fri Mar 11 15:03:40 2016 +0900

----------------------------------------------------------------------
 .../main/java/org/apache/hadoop/ipc/Client.java | 33 ++++++----
 .../src/main/resources/core-default.xml         |  9 ++-
 .../java/org/apache/hadoop/ipc/TestRPC.java     | 68 ++++++++++++++++++++
 3 files changed, 91 insertions(+), 19 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/682adc6b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Client.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Client.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Client.java
index 8d87957..3ae1d67 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Client.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Client.java
@@ -386,7 +386,7 @@ public class Client {
     private Socket socket = null;                 // connected socket
     private DataInputStream in;
     private DataOutputStream out;
-    private int rpcTimeout;
+    private final int rpcTimeout;
     private int maxIdleTime; //connections will be culled if it was idle for 
     //maxIdleTime msecs
     private final RetryPolicy connectionRetryPolicy;
@@ -394,8 +394,9 @@ public class Client {
     private int maxRetriesOnSocketTimeouts;
     private final boolean tcpNoDelay; // if T then disable Nagle's Algorithm
     private final boolean tcpLowLatency; // if T then use low-delay QoS
-    private boolean doPing; //do we need to send ping message
-    private int pingInterval; // how often sends ping to the server in msecs
+    private final boolean doPing; //do we need to send ping message
+    private final int pingInterval; // how often sends ping to the server
+    private final int soTimeout; // used by ipc ping and rpc timeout
     private ByteArrayOutputStream pingRequest; // ping message
     
     // currently active calls
@@ -434,6 +435,9 @@ public class Client {
         pingHeader.writeDelimitedTo(pingRequest);
       }
       this.pingInterval = remoteId.getPingInterval();
+      this.soTimeout =
+          (rpcTimeout == 0 || (doPing && pingInterval < rpcTimeout))?
+              this.pingInterval : this.rpcTimeout;
       this.serviceClass = serviceClass;
       if (LOG.isDebugEnabled()) {
         LOG.debug("The ping interval is " + this.pingInterval + " ms.");
@@ -484,12 +488,12 @@ public class Client {
 
       /* Process timeout exception
        * if the connection is not going to be closed or 
-       * is not configured to have a RPC timeout, send a ping.
-       * (if rpcTimeout is not set to be 0, then RPC should timeout.
-       * otherwise, throw the timeout exception.
+       * the RPC is not timed out yet, send a ping.
        */
-      private void handleTimeout(SocketTimeoutException e) throws IOException {
-        if (shouldCloseConnection.get() || !running.get() || rpcTimeout > 0) {
+      private void handleTimeout(SocketTimeoutException e, int waiting)
+          throws IOException {
+        if (shouldCloseConnection.get() || !running.get() ||
+            (0 < rpcTimeout && rpcTimeout <= waiting)) {
           throw e;
         } else {
           sendPing();
@@ -503,11 +507,13 @@ public class Client {
        */
       @Override
       public int read() throws IOException {
+        int waiting = 0;
         do {
           try {
             return super.read();
           } catch (SocketTimeoutException e) {
-            handleTimeout(e);
+            waiting += soTimeout;
+            handleTimeout(e, waiting);
           }
         } while (true);
       }
@@ -520,11 +526,13 @@ public class Client {
        */
       @Override
       public int read(byte[] buf, int off, int len) throws IOException {
+        int waiting = 0;
         do {
           try {
             return super.read(buf, off, len);
           } catch (SocketTimeoutException e) {
-            handleTimeout(e);
+            waiting += soTimeout;
+            handleTimeout(e, waiting);
           }
         } while (true);
       }
@@ -632,10 +640,7 @@ public class Client {
           }
           
           NetUtils.connect(this.socket, server, connectionTimeout);
-          if (rpcTimeout > 0) {
-            pingInterval = rpcTimeout;  // rpcTimeout overwrites pingInterval
-          }
-          this.socket.setSoTimeout(pingInterval);
+          this.socket.setSoTimeout(soTimeout);
           return;
         } catch (ConnectTimeoutException toe) {
           /* Check for an address change and update the local reference.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/682adc6b/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml b/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml
index 187f923..5037113 100644
--- a/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml
+++ b/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml
@@ -1054,7 +1054,7 @@
   <value>true</value>
   <description>Send a ping to the server when timeout on reading the response,
   if set to true. If no failure is detected, the client retries until at least
-  a byte is read.
+  a byte is read or the time given by ipc.client.rpc-timeout.ms is passed.
   </description>
 </property>
 
@@ -1071,10 +1071,9 @@
   <name>ipc.client.rpc-timeout.ms</name>
   <value>0</value>
   <description>Timeout on waiting response from server, in milliseconds.
-  Currently this timeout works only when ipc.client.ping is set to true
-  because it uses the same facilities with IPC ping.
-  The timeout overrides the ipc.ping.interval and client will throw exception
-  instead of sending ping when the interval is passed.
+  If ipc.client.ping is set to true and this rpc-timeout is greater than
+  the value of ipc.ping.interval, the effective value of the rpc-timeout is
+  rounded up to multiple of ipc.ping.interval.
   </description>
 </property>
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/682adc6b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestRPC.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestRPC.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestRPC.java
index 99bfc61..929b82b 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestRPC.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestRPC.java
@@ -1043,6 +1043,74 @@ public class TestRPC extends TestRpcBase {
     }
   }
 
+  /**
+   *  Test RPC timeout when ipc.client.ping is false.
+   */
+  @Test(timeout=30000)
+  public void testClientRpcTimeoutWithoutPing() throws Exception {
+    final Server server = new RPC.Builder(conf)
+        .setProtocol(TestProtocol.class).setInstance(new TestImpl())
+        .setBindAddress(ADDRESS).setPort(0)
+        .setQueueSizePerHandler(1).setNumHandlers(1).setVerbose(true)
+        .build();
+    server.start();
+
+    final Configuration conf = new Configuration();
+    conf.setBoolean(CommonConfigurationKeys.IPC_CLIENT_PING_KEY, false);
+    conf.setInt(CommonConfigurationKeys.IPC_CLIENT_RPC_TIMEOUT_KEY, 1000);
+    final TestProtocol proxy =
+        RPC.getProxy(TestProtocol.class, TestProtocol.versionID,
+            NetUtils.getConnectAddress(server), conf);
+
+    try {
+      proxy.sleep(3000);
+      fail("RPC should time out.");
+    } catch (SocketTimeoutException e) {
+      LOG.info("got expected timeout.", e);
+    } finally {
+      server.stop();
+      RPC.stopProxy(proxy);
+    }
+  }
+
+  /**
+   *  Test RPC timeout greater than ipc.ping.interval.
+   */
+  @Test(timeout=30000)
+  public void testClientRpcTimeoutGreaterThanPingInterval() throws Exception {
+    final Server server = new RPC.Builder(conf)
+        .setProtocol(TestProtocol.class).setInstance(new TestImpl())
+        .setBindAddress(ADDRESS).setPort(0)
+        .setQueueSizePerHandler(1).setNumHandlers(1).setVerbose(true)
+        .build();
+    server.start();
+
+    final Configuration conf = new Configuration();
+    conf.setBoolean(CommonConfigurationKeys.IPC_CLIENT_PING_KEY, true);
+    conf.setInt(CommonConfigurationKeys.IPC_PING_INTERVAL_KEY, 800);
+    conf.setInt(CommonConfigurationKeys.IPC_CLIENT_RPC_TIMEOUT_KEY, 1000);
+    final TestProtocol proxy =
+        RPC.getProxy(TestProtocol.class, TestProtocol.versionID,
+            NetUtils.getConnectAddress(server), conf);
+
+    // should not time out.
+    proxy.sleep(300);
+
+    // should not time out because effective rpc-timeout is
+    // multiple of ping interval: 1600 (= 800 * (1000 / 800 + 1))
+    proxy.sleep(1300);
+
+    try {
+      proxy.sleep(2000);
+      fail("RPC should time out.");
+    } catch (SocketTimeoutException e) {
+      LOG.info("got expected timeout.", e);
+    } finally {
+      server.stop();
+      RPC.stopProxy(proxy);
+    }
+  }
+
   public static void main(String[] args) throws Exception {
     new TestRPC().testCallsInternal(conf);
   }


[27/46] hadoop git commit: HDFS-9857. Erasure Coding: Rename replication-based names in BlockManager to more generic [part-1]. Contributed by Rakesh R.

Posted by ae...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/32d043d9/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestPendingReplication.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestPendingReplication.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestPendingReplication.java
index 18f28d5..f04387d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestPendingReplication.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestPendingReplication.java
@@ -183,7 +183,7 @@ public class TestPendingReplication {
 
       PendingReplicationBlocks pendingReplications =
           blkManager.pendingReplications;
-      UnderReplicatedBlocks neededReplications = blkManager.neededReplications;
+      LowRedundancyBlocks neededReconstruction = blkManager.neededReconstruction;
       BlocksMap blocksMap = blkManager.blocksMap;
 
       //
@@ -227,9 +227,9 @@ public class TestPendingReplication {
       }
 
       //
-      // Verify that block moves to neededReplications
+      // Verify that block moves to neededReconstruction
       //
-      while (neededReplications.size() == 0) {
+      while (neededReconstruction.size() == 0) {
         try {
           Thread.sleep(100);
         } catch (Exception e) {
@@ -238,14 +238,14 @@ public class TestPendingReplication {
 
       // Verify that the generation stamp we will try to replicate
       // is now 1
-      for (Block b: neededReplications) {
+      for (Block b: neededReconstruction) {
         assertEquals("Generation stamp is 1 ", 1,
             b.getGenerationStamp());
       }
 
-      // Verify size of neededReplications is exactly 1.
-      assertEquals("size of neededReplications is 1 ", 1,
-          neededReplications.size());
+      // Verify size of neededReconstruction is exactly 1.
+      assertEquals("size of neededReconstruction is 1 ", 1,
+          neededReconstruction.size());
     } finally {
       if (cluster != null) {
         cluster.shutdown();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/32d043d9/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestReplicationPolicy.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestReplicationPolicy.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestReplicationPolicy.java
index be63d87..9f8985a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestReplicationPolicy.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestReplicationPolicy.java
@@ -836,12 +836,12 @@ public class TestReplicationPolicy extends BaseReplicationPolicyTest {
         .format(true).build();
     try {
       cluster.waitActive();
-      final UnderReplicatedBlocks neededReplications = cluster.getNameNode()
-          .getNamesystem().getBlockManager().neededReplications;
+      final LowRedundancyBlocks neededReconstruction = cluster.getNameNode()
+          .getNamesystem().getBlockManager().neededReconstruction;
       for (int i = 0; i < 100; i++) {
         // Adding the blocks directly to normal priority
 
-        neededReplications.add(genBlockInfo(ThreadLocalRandom.current().
+        neededReconstruction.add(genBlockInfo(ThreadLocalRandom.current().
             nextLong()), 2, 0, 0, 3);
       }
       // Lets wait for the replication interval, to start process normal
@@ -849,7 +849,7 @@ public class TestReplicationPolicy extends BaseReplicationPolicyTest {
       Thread.sleep(DFS_NAMENODE_REPLICATION_INTERVAL);
       
       // Adding the block directly to high priority list
-      neededReplications.add(genBlockInfo(ThreadLocalRandom.current().
+      neededReconstruction.add(genBlockInfo(ThreadLocalRandom.current().
           nextLong()), 1, 0, 0, 3);
 
       // Lets wait for the replication interval
@@ -858,68 +858,68 @@ public class TestReplicationPolicy extends BaseReplicationPolicyTest {
       // Check replication completed successfully. Need not wait till it process
       // all the 100 normal blocks.
       assertFalse("Not able to clear the element from high priority list",
-          neededReplications.iterator(HIGH_PRIORITY).hasNext());
+          neededReconstruction.iterator(HIGH_PRIORITY).hasNext());
     } finally {
       cluster.shutdown();
     }
   }
   
   /**
-   * Test for the ChooseUnderReplicatedBlocks are processed based on priority
+   * Test for the ChooseLowRedundancyBlocks are processed based on priority
    */
   @Test
-  public void testChooseUnderReplicatedBlocks() throws Exception {
-    UnderReplicatedBlocks underReplicatedBlocks = new UnderReplicatedBlocks();
+  public void testChooseLowRedundancyBlocks() throws Exception {
+    LowRedundancyBlocks lowRedundancyBlocks = new LowRedundancyBlocks();
 
     for (int i = 0; i < 5; i++) {
       // Adding QUEUE_HIGHEST_PRIORITY block
-      underReplicatedBlocks.add(genBlockInfo(ThreadLocalRandom.current().
+      lowRedundancyBlocks.add(genBlockInfo(ThreadLocalRandom.current().
           nextLong()), 1, 0, 0, 3);
 
-      // Adding QUEUE_VERY_UNDER_REPLICATED block
-      underReplicatedBlocks.add(genBlockInfo(ThreadLocalRandom.current().
+      // Adding QUEUE_VERY_LOW_REDUNDANCY block
+      lowRedundancyBlocks.add(genBlockInfo(ThreadLocalRandom.current().
           nextLong()), 2, 0, 0, 7);
 
       // Adding QUEUE_REPLICAS_BADLY_DISTRIBUTED block
-      underReplicatedBlocks.add(genBlockInfo(ThreadLocalRandom.current().
+      lowRedundancyBlocks.add(genBlockInfo(ThreadLocalRandom.current().
           nextLong()), 6, 0, 0, 6);
 
-      // Adding QUEUE_UNDER_REPLICATED block
-      underReplicatedBlocks.add(genBlockInfo(ThreadLocalRandom.current().
+      // Adding QUEUE_LOW_REDUNDANCY block
+      lowRedundancyBlocks.add(genBlockInfo(ThreadLocalRandom.current().
           nextLong()), 5, 0, 0, 6);
 
       // Adding QUEUE_WITH_CORRUPT_BLOCKS block
-      underReplicatedBlocks.add(genBlockInfo(ThreadLocalRandom.current().
+      lowRedundancyBlocks.add(genBlockInfo(ThreadLocalRandom.current().
           nextLong()), 0, 0, 0, 3);
     }
 
-    // Choose 6 blocks from UnderReplicatedBlocks. Then it should pick 5 blocks
-    // from
-    // QUEUE_HIGHEST_PRIORITY and 1 block from QUEUE_VERY_UNDER_REPLICATED.
+    // Choose 6 blocks from lowRedundancyBlocks. Then it should pick 5 blocks
+    // from QUEUE_HIGHEST_PRIORITY and 1 block from QUEUE_VERY_LOW_REDUNDANCY.
     List<List<BlockInfo>> chosenBlocks =
-        underReplicatedBlocks.chooseUnderReplicatedBlocks(6);
+        lowRedundancyBlocks.chooseLowRedundancyBlocks(6);
     assertTheChosenBlocks(chosenBlocks, 5, 1, 0, 0, 0);
 
-    // Choose 10 blocks from UnderReplicatedBlocks. Then it should pick 4 blocks from
-    // QUEUE_VERY_UNDER_REPLICATED, 5 blocks from QUEUE_UNDER_REPLICATED and 1
+    // Choose 10 blocks from lowRedundancyBlocks. Then it should pick 4 blocks
+    // from QUEUE_VERY_LOW_REDUNDANCY, 5 blocks from QUEUE_LOW_REDUNDANCY and 1
     // block from QUEUE_REPLICAS_BADLY_DISTRIBUTED.
-    chosenBlocks = underReplicatedBlocks.chooseUnderReplicatedBlocks(10);
+    chosenBlocks = lowRedundancyBlocks.chooseLowRedundancyBlocks(10);
     assertTheChosenBlocks(chosenBlocks, 0, 4, 5, 1, 0);
 
     // Adding QUEUE_HIGHEST_PRIORITY
-    underReplicatedBlocks.add(genBlockInfo(ThreadLocalRandom.current().
+    lowRedundancyBlocks.add(genBlockInfo(ThreadLocalRandom.current().
         nextLong()), 0, 1, 0, 3);
 
-    // Choose 10 blocks from UnderReplicatedBlocks. Then it should pick 1 block from
-    // QUEUE_HIGHEST_PRIORITY, 4 blocks from QUEUE_REPLICAS_BADLY_DISTRIBUTED
-    chosenBlocks = underReplicatedBlocks.chooseUnderReplicatedBlocks(10);
+    // Choose 10 blocks from lowRedundancyBlocks. Then it should pick 1 block
+    // from QUEUE_HIGHEST_PRIORITY, 4 blocks from
+    // QUEUE_REPLICAS_BADLY_DISTRIBUTED
+    chosenBlocks = lowRedundancyBlocks.chooseLowRedundancyBlocks(10);
     assertTheChosenBlocks(chosenBlocks, 1, 0, 0, 4);
 
     // Since it is reached to end of all lists,
     // should start picking the blocks from start.
-    // Choose 7 blocks from UnderReplicatedBlocks. Then it should pick 6 blocks from
-    // QUEUE_HIGHEST_PRIORITY, 1 block from QUEUE_VERY_UNDER_REPLICATED.
-    chosenBlocks = underReplicatedBlocks.chooseUnderReplicatedBlocks(7);
+    // Choose 7 blocks from lowRedundancyBlocks. Then it should pick 6 blocks
+    // from QUEUE_HIGHEST_PRIORITY, 1 block from QUEUE_VERY_LOW_REDUNDANCY.
+    chosenBlocks = lowRedundancyBlocks.chooseLowRedundancyBlocks(7);
     assertTheChosenBlocks(chosenBlocks, 6, 1, 0, 0, 0);
   }
   
@@ -1268,45 +1268,45 @@ public class TestReplicationPolicy extends BaseReplicationPolicyTest {
 
   @Test(timeout = 60000)
   public void testUpdateDoesNotCauseSkippedReplication() {
-    UnderReplicatedBlocks underReplicatedBlocks = new UnderReplicatedBlocks();
+    LowRedundancyBlocks lowRedundancyBlocks = new LowRedundancyBlocks();
 
     BlockInfo block1 = genBlockInfo(ThreadLocalRandom.current().nextLong());
     BlockInfo block2 = genBlockInfo(ThreadLocalRandom.current().nextLong());
     BlockInfo block3 = genBlockInfo(ThreadLocalRandom.current().nextLong());
 
-    // Adding QUEUE_VERY_UNDER_REPLICATED block
+    // Adding QUEUE_VERY_LOW_REDUNDANCY block
     final int block1CurReplicas = 2;
     final int block1ExpectedReplicas = 7;
-    underReplicatedBlocks.add(block1, block1CurReplicas, 0, 0,
+    lowRedundancyBlocks.add(block1, block1CurReplicas, 0, 0,
         block1ExpectedReplicas);
 
-    // Adding QUEUE_VERY_UNDER_REPLICATED block
-    underReplicatedBlocks.add(block2, 2, 0, 0, 7);
+    // Adding QUEUE_VERY_LOW_REDUNDANCY block
+    lowRedundancyBlocks.add(block2, 2, 0, 0, 7);
 
-    // Adding QUEUE_UNDER_REPLICATED block
-    underReplicatedBlocks.add(block3, 2, 0, 0, 6);
+    // Adding QUEUE_LOW_REDUNDANCY block
+    lowRedundancyBlocks.add(block3, 2, 0, 0, 6);
 
     List<List<BlockInfo>> chosenBlocks;
 
-    // Choose 1 block from UnderReplicatedBlocks. Then it should pick 1 block
-    // from QUEUE_VERY_UNDER_REPLICATED.
-    chosenBlocks = underReplicatedBlocks.chooseUnderReplicatedBlocks(1);
+    // Choose 1 block from lowRedundancyBlocks. Then it should pick 1 block
+    // from QUEUE_VERY_LOW_REDUNDANCY.
+    chosenBlocks = lowRedundancyBlocks.chooseLowRedundancyBlocks(1);
     assertTheChosenBlocks(chosenBlocks, 0, 1, 0, 0, 0);
 
     // Increasing the replications will move the block down a
     // priority.  This simulates a replica being completed in between checks.
-    underReplicatedBlocks.update(block1, block1CurReplicas+1, 0, 0,
+    lowRedundancyBlocks.update(block1, block1CurReplicas+1, 0, 0,
         block1ExpectedReplicas, 1, 0);
 
-    // Choose 1 block from UnderReplicatedBlocks. Then it should pick 1 block
-    // from QUEUE_VERY_UNDER_REPLICATED.
+    // Choose 1 block from lowRedundancyBlocks. Then it should pick 1 block
+    // from QUEUE_VERY_LOW_REDUNDANCY.
     // This block was moved up a priority and should not be skipped over.
-    chosenBlocks = underReplicatedBlocks.chooseUnderReplicatedBlocks(1);
+    chosenBlocks = lowRedundancyBlocks.chooseLowRedundancyBlocks(1);
     assertTheChosenBlocks(chosenBlocks, 0, 1, 0, 0, 0);
 
-    // Choose 1 block from UnderReplicatedBlocks. Then it should pick 1 block
-    // from QUEUE_UNDER_REPLICATED.
-    chosenBlocks = underReplicatedBlocks.chooseUnderReplicatedBlocks(1);
+    // Choose 1 block from lowRedundancyBlocks. Then it should pick 1 block
+    // from QUEUE_LOW_REDUNDANCY.
+    chosenBlocks = lowRedundancyBlocks.chooseLowRedundancyBlocks(1);
     assertTheChosenBlocks(chosenBlocks, 0, 0, 1, 0, 0);
   }
 
@@ -1317,27 +1317,27 @@ public class TestReplicationPolicy extends BaseReplicationPolicyTest {
     when(mockNS.hasWriteLock()).thenReturn(true);
     when(mockNS.hasReadLock()).thenReturn(true);
     BlockManager bm = new BlockManager(mockNS, false, new HdfsConfiguration());
-    UnderReplicatedBlocks underReplicatedBlocks = bm.neededReplications;
+    LowRedundancyBlocks lowRedundancyBlocks = bm.neededReconstruction;
 
     BlockInfo block1 = genBlockInfo(ThreadLocalRandom.current().nextLong());
     BlockInfo block2 = genBlockInfo(ThreadLocalRandom.current().nextLong());
 
-    // Adding QUEUE_UNDER_REPLICATED block
-    underReplicatedBlocks.add(block1, 0, 0, 1, 1);
+    // Adding QUEUE_LOW_REDUNDANCY block
+    lowRedundancyBlocks.add(block1, 0, 0, 1, 1);
 
-    // Adding QUEUE_UNDER_REPLICATED block
-    underReplicatedBlocks.add(block2, 0, 0, 1, 1);
+    // Adding QUEUE_LOW_REDUNDANCY block
+    lowRedundancyBlocks.add(block2, 0, 0, 1, 1);
 
     List<List<BlockInfo>> chosenBlocks;
 
-    // Choose 1 block from UnderReplicatedBlocks. Then it should pick 1 block
-    // from QUEUE_VERY_UNDER_REPLICATED.
-    chosenBlocks = underReplicatedBlocks.chooseUnderReplicatedBlocks(1);
+    // Choose 1 block from lowRedundancyBlocks. Then it should pick 1 block
+    // from QUEUE_VERY_LOW_REDUNDANCY.
+    chosenBlocks = lowRedundancyBlocks.chooseLowRedundancyBlocks(1);
     assertTheChosenBlocks(chosenBlocks, 1, 0, 0, 0, 0);
 
-    // Adding this block collection to the BlockManager, so that when we add the
+    // Adding this block collection to the BlockManager, so that when we add
     // block under construction, the BlockManager will realize the expected
-    // replication has been achieved and remove it from the under-replicated
+    // replication has been achieved and remove it from the low redundancy
     // queue.
     BlockInfoContiguous info = new BlockInfoContiguous(block1, (short) 1);
     info.convertToBlockUnderConstruction(BlockUCState.UNDER_CONSTRUCTION, null);
@@ -1353,9 +1353,9 @@ public class TestReplicationPolicy extends BaseReplicationPolicyTest {
         ReplicaState.FINALIZED), storages[0]);
 
     // Choose 1 block from UnderReplicatedBlocks. Then it should pick 1 block
-    // from QUEUE_VERY_UNDER_REPLICATED.
+    // from QUEUE_VERY_LOW_REDUNDANCY.
     // This block remains and should not be skipped over.
-    chosenBlocks = underReplicatedBlocks.chooseUnderReplicatedBlocks(1);
+    chosenBlocks = lowRedundancyBlocks.chooseLowRedundancyBlocks(1);
     assertTheChosenBlocks(chosenBlocks, 1, 0, 0, 0, 0);
   }
 
@@ -1367,7 +1367,7 @@ public class TestReplicationPolicy extends BaseReplicationPolicyTest {
     when(mockNS.hasWriteLock()).thenReturn(true);
 
     BlockManager bm = new BlockManager(mockNS, false, new HdfsConfiguration());
-    UnderReplicatedBlocks underReplicatedBlocks = bm.neededReplications;
+    LowRedundancyBlocks lowRedundancyBlocks = bm.neededReconstruction;
 
     long blkID1 = ThreadLocalRandom.current().nextLong();
     if (blkID1 < 0) {
@@ -1381,17 +1381,17 @@ public class TestReplicationPolicy extends BaseReplicationPolicyTest {
     BlockInfo block1 = genBlockInfo(blkID1);
     BlockInfo block2 = genBlockInfo(blkID2);
 
-    // Adding QUEUE_UNDER_REPLICATED block
-    underReplicatedBlocks.add(block1, 0, 0, 1, 1);
+    // Adding QUEUE_LOW_REDUNDANCY block
+    lowRedundancyBlocks.add(block1, 0, 0, 1, 1);
 
-    // Adding QUEUE_UNDER_REPLICATED block
-    underReplicatedBlocks.add(block2, 0, 0, 1, 1);
+    // Adding QUEUE_LOW_REDUNDANCY block
+    lowRedundancyBlocks.add(block2, 0, 0, 1, 1);
 
     List<List<BlockInfo>> chosenBlocks;
 
-    // Choose 1 block from UnderReplicatedBlocks. Then it should pick 1 block
-    // from QUEUE_VERY_UNDER_REPLICATED.
-    chosenBlocks = underReplicatedBlocks.chooseUnderReplicatedBlocks(1);
+    // Choose 1 block from lowRedundancyBlocks. Then it should pick 1 block
+    // from QUEUE_VERY_LOW_REDUNDANCY.
+    chosenBlocks = lowRedundancyBlocks.chooseLowRedundancyBlocks(1);
     assertTheChosenBlocks(chosenBlocks, 1, 0, 0, 0, 0);
 
     final BlockInfoContiguous info = new BlockInfoContiguous(block1, (short) 1);
@@ -1425,10 +1425,10 @@ public class TestReplicationPolicy extends BaseReplicationPolicyTest {
 
     bm.convertLastBlockToUnderConstruction(mbc, 0L);
 
-    // Choose 1 block from UnderReplicatedBlocks. Then it should pick 1 block
-    // from QUEUE_VERY_UNDER_REPLICATED.
+    // Choose 1 block from lowRedundancyBlocks. Then it should pick 1 block
+    // from QUEUE_VERY_LOW_REDUNDANCY.
     // This block remains and should not be skipped over.
-    chosenBlocks = underReplicatedBlocks.chooseUnderReplicatedBlocks(1);
+    chosenBlocks = lowRedundancyBlocks.chooseLowRedundancyBlocks(1);
     assertTheChosenBlocks(chosenBlocks, 1, 0, 0, 0, 0);
   }
 
@@ -1439,30 +1439,30 @@ public class TestReplicationPolicy extends BaseReplicationPolicyTest {
     when(mockNS.hasReadLock()).thenReturn(true);
 
     BlockManager bm = new BlockManager(mockNS, false, new HdfsConfiguration());
-    UnderReplicatedBlocks underReplicatedBlocks = bm.neededReplications;
+    LowRedundancyBlocks lowRedundancyBlocks = bm.neededReconstruction;
 
     BlockInfo block1 = genBlockInfo(ThreadLocalRandom.current().nextLong());
     BlockInfo block2 = genBlockInfo(ThreadLocalRandom.current().nextLong());
 
-    // Adding QUEUE_UNDER_REPLICATED block
-    underReplicatedBlocks.add(block1, 0, 0, 1, 1);
+    // Adding QUEUE_LOW_REDUNDANCY block
+    lowRedundancyBlocks.add(block1, 0, 0, 1, 1);
 
-    // Adding QUEUE_UNDER_REPLICATED block
-    underReplicatedBlocks.add(block2, 0, 0, 1, 1);
+    // Adding QUEUE_LOW_REDUNDANCY block
+    lowRedundancyBlocks.add(block2, 0, 0, 1, 1);
 
     List<List<BlockInfo>> chosenBlocks;
 
-    // Choose 1 block from UnderReplicatedBlocks. Then it should pick 1 block
-    // from QUEUE_VERY_UNDER_REPLICATED.
-    chosenBlocks = underReplicatedBlocks.chooseUnderReplicatedBlocks(1);
+    // Choose 1 block from lowRedundancyBlocks. Then it should pick 1 block
+    // from QUEUE_VERY_LOW_REDUNDANCY.
+    chosenBlocks = lowRedundancyBlocks.chooseLowRedundancyBlocks(1);
     assertTheChosenBlocks(chosenBlocks, 1, 0, 0, 0, 0);
 
     bm.setReplication((short)0, (short)1, block1);
 
     // Choose 1 block from UnderReplicatedBlocks. Then it should pick 1 block
-    // from QUEUE_VERY_UNDER_REPLICATED.
+    // from QUEUE_VERY_LOW_REDUNDANCY.
     // This block remains and should not be skipped over.
-    chosenBlocks = underReplicatedBlocks.chooseUnderReplicatedBlocks(1);
+    chosenBlocks = lowRedundancyBlocks.chooseLowRedundancyBlocks(1);
     assertTheChosenBlocks(chosenBlocks, 1, 0, 0, 0, 0);
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/32d043d9/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestUnderReplicatedBlockQueues.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestUnderReplicatedBlockQueues.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestUnderReplicatedBlockQueues.java
deleted file mode 100644
index b228d09..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestUnderReplicatedBlockQueues.java
+++ /dev/null
@@ -1,182 +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.server.blockmanagement;
-
-import java.util.Iterator;
-
-import org.apache.hadoop.hdfs.protocol.Block;
-import org.apache.hadoop.hdfs.server.namenode.ErasureCodingPolicyManager;
-import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
-import org.junit.Test;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.fail;
-
-public class TestUnderReplicatedBlockQueues {
-
-  private final ErasureCodingPolicy ecPolicy =
-      ErasureCodingPolicyManager.getSystemDefaultPolicy();
-
-  private BlockInfo genBlockInfo(long id) {
-    return new BlockInfoContiguous(new Block(id), (short) 3);
-  }
-
-  private BlockInfo genStripedBlockInfo(long id, long numBytes) {
-    BlockInfoStriped sblk =  new BlockInfoStriped(new Block(id), ecPolicy);
-    sblk.setNumBytes(numBytes);
-    return sblk;
-  }
-
-  /**
-   * Test that adding blocks with different replication counts puts them
-   * into different queues
-   * @throws Throwable if something goes wrong
-   */
-  @Test
-  public void testBlockPriorities() throws Throwable {
-    UnderReplicatedBlocks queues = new UnderReplicatedBlocks();
-    BlockInfo block1 = genBlockInfo(1);
-    BlockInfo block2 = genBlockInfo(2);
-    BlockInfo block_very_under_replicated = genBlockInfo(3);
-    BlockInfo block_corrupt = genBlockInfo(4);
-    BlockInfo block_corrupt_repl_one = genBlockInfo(5);
-
-    //add a block with a single entry
-    assertAdded(queues, block1, 1, 0, 3);
-
-    assertEquals(1, queues.getUnderReplicatedBlockCount());
-    assertEquals(1, queues.size());
-    assertInLevel(queues, block1, UnderReplicatedBlocks.QUEUE_HIGHEST_PRIORITY);
-    //repeated additions fail
-    assertFalse(queues.add(block1, 1, 0, 0, 3));
-
-    //add a second block with two replicas
-    assertAdded(queues, block2, 2, 0, 3);
-    assertEquals(2, queues.getUnderReplicatedBlockCount());
-    assertEquals(2, queues.size());
-    assertInLevel(queues, block2, UnderReplicatedBlocks.QUEUE_UNDER_REPLICATED);
-    //now try to add a block that is corrupt
-    assertAdded(queues, block_corrupt, 0, 0, 3);
-    assertEquals(3, queues.size());
-    assertEquals(2, queues.getUnderReplicatedBlockCount());
-    assertEquals(1, queues.getCorruptBlockSize());
-    assertInLevel(queues, block_corrupt,
-                  UnderReplicatedBlocks.QUEUE_WITH_CORRUPT_BLOCKS);
-
-    //insert a very under-replicated block
-    assertAdded(queues, block_very_under_replicated, 4, 0, 25);
-    assertInLevel(queues, block_very_under_replicated,
-                  UnderReplicatedBlocks.QUEUE_VERY_UNDER_REPLICATED);
-
-    //insert a corrupt block with replication factor 1
-    assertAdded(queues, block_corrupt_repl_one, 0, 0, 1);
-    assertEquals(2, queues.getCorruptBlockSize());
-    assertEquals(1, queues.getCorruptReplOneBlockSize());
-    queues.update(block_corrupt_repl_one, 0, 0, 0, 3, 0, 2);
-    assertEquals(0, queues.getCorruptReplOneBlockSize());
-    queues.update(block_corrupt, 0, 0, 0, 1, 0, -2);
-    assertEquals(1, queues.getCorruptReplOneBlockSize());
-    queues.update(block_very_under_replicated, 0, 0, 0, 1, -4, -24);
-    assertEquals(2, queues.getCorruptReplOneBlockSize());
-  }
-
-  @Test
-  public void testStripedBlockPriorities() throws Throwable {
-    int dataBlkNum = ecPolicy.getNumDataUnits();
-    int parityBlkNUm = ecPolicy.getNumParityUnits();
-    doTestStripedBlockPriorities(1, parityBlkNUm);
-    doTestStripedBlockPriorities(dataBlkNum, parityBlkNUm);
-  }
-
-  private void doTestStripedBlockPriorities(int dataBlkNum, int parityBlkNum)
-      throws Throwable {
-    int groupSize = dataBlkNum + parityBlkNum;
-    long numBytes = ecPolicy.getCellSize() * dataBlkNum;
-    UnderReplicatedBlocks queues = new UnderReplicatedBlocks();
-    int numUR = 0;
-    int numCorrupt = 0;
-
-    // add under replicated blocks
-    for (int i = 0; dataBlkNum + i < groupSize; i++) {
-      BlockInfo block = genStripedBlockInfo(-100 - 100 * i, numBytes);
-      assertAdded(queues, block, dataBlkNum + i, 0, groupSize);
-      numUR++;
-      assertEquals(numUR, queues.getUnderReplicatedBlockCount());
-      assertEquals(numUR + numCorrupt, queues.size());
-      if (i == 0) {
-        assertInLevel(queues, block,
-            UnderReplicatedBlocks.QUEUE_HIGHEST_PRIORITY);
-      } else if (i * 3 < parityBlkNum + 1) {
-        assertInLevel(queues, block,
-            UnderReplicatedBlocks.QUEUE_VERY_UNDER_REPLICATED);
-      } else {
-        assertInLevel(queues, block,
-            UnderReplicatedBlocks.QUEUE_UNDER_REPLICATED);
-      }
-    }
-
-    // add a corrupted block
-    BlockInfo block_corrupt = genStripedBlockInfo(-10, numBytes);
-    assertEquals(numCorrupt, queues.getCorruptBlockSize());
-    assertAdded(queues, block_corrupt, dataBlkNum - 1, 0, groupSize);
-    numCorrupt++;
-    assertEquals(numUR + numCorrupt, queues.size());
-    assertEquals(numUR, queues.getUnderReplicatedBlockCount());
-    assertEquals(numCorrupt, queues.getCorruptBlockSize());
-    assertInLevel(queues, block_corrupt,
-        UnderReplicatedBlocks.QUEUE_WITH_CORRUPT_BLOCKS);
-  }
-
-  private void assertAdded(UnderReplicatedBlocks queues,
-                           BlockInfo block,
-                           int curReplicas,
-                           int decomissionedReplicas,
-                           int expectedReplicas) {
-    assertTrue("Failed to add " + block,
-               queues.add(block,
-                          curReplicas, 0,
-                          decomissionedReplicas,
-                          expectedReplicas));
-  }
-
-  /**
-   * Determine whether or not a block is in a level without changing the API.
-   * Instead get the per-level iterator and run though it looking for a match.
-   * If the block is not found, an assertion is thrown.
-   *
-   * This is inefficient, but this is only a test case.
-   * @param queues queues to scan
-   * @param block block to look for
-   * @param level level to select
-   */
-  private void assertInLevel(UnderReplicatedBlocks queues,
-                             Block block,
-                             int level) {
-    final Iterator<BlockInfo> bi = queues.iterator(level);
-    while (bi.hasNext()) {
-      Block next = bi.next();
-      if (block.equals(next)) {
-        return;
-      }
-    }
-    fail("Block " + block + " not found in level " + level);
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/32d043d9/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestMetaSave.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestMetaSave.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestMetaSave.java
index 690812f..058ab8a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestMetaSave.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestMetaSave.java
@@ -152,7 +152,7 @@ public class TestMetaSave {
       line = reader.readLine();
       assertTrue(line.equals("Dead Datanodes: 1"));
       line = reader.readLine();
-      assertTrue(line.equals("Metasave: Blocks waiting for replication: 0"));
+      assertTrue(line.equals("Metasave: Blocks waiting for reconstruction: 0"));
       line = reader.readLine();
       assertTrue(line.equals("Mis-replicated blocks that have been postponed:"));
       line = reader.readLine();


[29/46] hadoop git commit: CapacityScheduler: Improve preemption to only kill containers that would satisfy the incoming request. (Wangda Tan)

Posted by ae...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/7e8c9beb/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 2694957..4441c6b 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
@@ -356,4 +356,40 @@ public class TestUtils {
     CapacityScheduler cs = (CapacityScheduler) rm.getResourceScheduler();
     return cs.getSchedulerApplications().get(appId).getCurrentAppAttempt();
   }
+
+  /**
+   * Get a queue structure:
+   * <pre>
+   *             Root
+   *            /  |  \
+   *           a   b   c
+   *          10   20  70
+   * </pre>
+   */
+  public static Configuration
+  getConfigurationWithMultipleQueues(Configuration config) {
+    CapacitySchedulerConfiguration conf =
+        new CapacitySchedulerConfiguration(config);
+
+    // Define top-level queues
+    conf.setQueues(CapacitySchedulerConfiguration.ROOT,
+        new String[] { "a", "b", "c" });
+
+    final String A = CapacitySchedulerConfiguration.ROOT + ".a";
+    conf.setCapacity(A, 10);
+    conf.setMaximumCapacity(A, 100);
+    conf.setUserLimitFactor(A, 100);
+
+    final String B = CapacitySchedulerConfiguration.ROOT + ".b";
+    conf.setCapacity(B, 20);
+    conf.setMaximumCapacity(B, 100);
+    conf.setUserLimitFactor(B, 100);
+
+    final String C = CapacitySchedulerConfiguration.ROOT + ".c";
+    conf.setCapacity(C, 70);
+    conf.setMaximumCapacity(C, 100);
+    conf.setUserLimitFactor(C, 100);
+
+    return conf;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7e8c9beb/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFairSchedulerPreemption.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/fair/TestFairSchedulerPreemption.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFairSchedulerPreemption.java
index 5bdcc08..2456594 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFairSchedulerPreemption.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFairSchedulerPreemption.java
@@ -1451,7 +1451,7 @@ public class TestFairSchedulerPreemption extends FairSchedulerTestBase {
 
     // Trigger container rescheduled event
     scheduler.handle(new ContainerPreemptEvent(appAttemptId, rmContainer,
-            SchedulerEventType.KILL_PREEMPTED_CONTAINER));
+            SchedulerEventType.MARK_CONTAINER_FOR_KILLABLE));
 
     List<ResourceRequest> requests = rmContainer.getResourceRequests();
     // Once recovered, resource request will be present again in app


[08/46] hadoop git commit: HADOOP-11996. Improve and restructure native ISAL support (Kai Zheng via cmccabe)

Posted by ae...@apache.org.
HADOOP-11996. Improve and restructure native ISAL support (Kai Zheng via cmccabe)


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

Branch: refs/heads/HDFS-7240
Commit: 658ee95ff367eaf58d76581e75bdaba0dd0d871e
Parents: 6e9a582
Author: Colin Patrick Mccabe <cm...@cloudera.com>
Authored: Fri Mar 11 12:56:12 2016 -0800
Committer: Colin Patrick Mccabe <cm...@cloudera.com>
Committed: Fri Mar 11 12:57:54 2016 -0800

----------------------------------------------------------------------
 .../hadoop-common/src/CMakeLists.txt            |  22 +-
 .../src/main/native/native.vcxproj              |  11 +-
 .../io/erasurecode/coder/erasure_code_native.c  |  49 ---
 ...he_hadoop_io_erasurecode_ErasureCodeNative.h |  29 --
 .../src/org/apache/hadoop/io/erasurecode/dump.c | 100 ++++++
 .../src/org/apache/hadoop/io/erasurecode/dump.h |  40 +++
 .../apache/hadoop/io/erasurecode/erasure_code.c | 239 +--------------
 .../apache/hadoop/io/erasurecode/erasure_code.h | 107 +++++++
 .../hadoop/io/erasurecode/erasure_coder.c       | 229 ++++++++++++++
 .../hadoop/io/erasurecode/erasure_coder.h       |  88 ++++++
 .../org/apache/hadoop/io/erasurecode/gf_util.c  |  54 ++++
 .../org/apache/hadoop/io/erasurecode/gf_util.h  | 111 +++++++
 .../io/erasurecode/include/erasure_code.h       | 125 --------
 .../hadoop/io/erasurecode/include/gf_util.h     | 111 -------
 .../apache/hadoop/io/erasurecode/isal_load.c    | 148 +++++++++
 .../apache/hadoop/io/erasurecode/isal_load.h    | 149 +++++++++
 .../apache/hadoop/io/erasurecode/jni_common.c   |  98 ++++++
 .../apache/hadoop/io/erasurecode/jni_common.h   |  42 +++
 .../io/erasurecode/jni_erasure_code_native.c    |  45 +++
 .../hadoop/io/erasurecode/jni_rs_decoder.c      |  72 +++++
 .../hadoop/io/erasurecode/jni_rs_encoder.c      |  66 ++++
 ...he_hadoop_io_erasurecode_ErasureCodeNative.h |  29 ++
 ...io_erasurecode_rawcoder_NativeRSRawDecoder.h |  37 +++
 ...io_erasurecode_rawcoder_NativeRSRawEncoder.h |  37 +++
 .../hadoop/io/erasurecode/erasure_code_test.c   | 307 ++++---------------
 25 files changed, 1545 insertions(+), 800 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/658ee95f/hadoop-common-project/hadoop-common/src/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/CMakeLists.txt b/hadoop-common-project/hadoop-common/src/CMakeLists.txt
index 51fb0fe..8026eb5 100644
--- a/hadoop-common-project/hadoop-common/src/CMakeLists.txt
+++ b/hadoop-common-project/hadoop-common/src/CMakeLists.txt
@@ -103,13 +103,25 @@ find_library(ISAL_LIBRARY
 set(CMAKE_FIND_LIBRARY_SUFFIXES ${STORED_CMAKE_FIND_LIBRARY_SUFFIXES})
 if (ISAL_LIBRARY)
     GET_FILENAME_COMPONENT(HADOOP_ISAL_LIBRARY ${ISAL_LIBRARY} NAME)
-    set(ISAL_INCLUDE_DIR ${SRC}/io/erasurecode/include)
+    set(ISAL_INCLUDE_DIR ${SRC}/io/erasurecode)
     set(ISAL_SOURCE_FILES
-        ${SRC}/io/erasurecode/coder/erasure_code_native.c
-        ${SRC}/io/erasurecode/erasure_code.c)
+        ${SRC}/io/erasurecode/isal_load.c
+        ${SRC}/io/erasurecode/erasure_code.c
+        ${SRC}/io/erasurecode/gf_util.c
+        ${SRC}/io/erasurecode/dump.c
+        ${SRC}/io/erasurecode/erasure_coder.c
+        ${SRC}/io/erasurecode/jni_erasure_code_native.c
+        ${SRC}/io/erasurecode/jni_common.c
+        ${SRC}/io/erasurecode/jni_rs_encoder.c
+        ${SRC}/io/erasurecode/jni_rs_decoder.c)
+
         add_executable(erasure_code_test
-             ${SRC}/io/erasurecode/erasure_code.c
-             ${TST}/io/erasurecode/erasure_code_test.c
+        ${SRC}/io/erasurecode/isal_load.c
+        ${SRC}/io/erasurecode/erasure_code.c
+        ${SRC}/io/erasurecode/gf_util.c
+        ${SRC}/io/erasurecode/dump.c
+        ${SRC}/io/erasurecode/erasure_coder.c
+        ${TST}/io/erasurecode/erasure_code_test.c
         )
         target_link_libraries(erasure_code_test ${CMAKE_DL_LIBS})
 else (ISAL_LIBRARY)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/658ee95f/hadoop-common-project/hadoop-common/src/main/native/native.vcxproj
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/native/native.vcxproj b/hadoop-common-project/hadoop-common/src/main/native/native.vcxproj
index 17149f7..2274c41 100644
--- a/hadoop-common-project/hadoop-common/src/main/native/native.vcxproj
+++ b/hadoop-common-project/hadoop-common/src/main/native/native.vcxproj
@@ -162,10 +162,17 @@
     <ClCompile Include="src\org\apache\hadoop\yarn\server\nodemanager\windows_secure_container_executor.c">
       <AdditionalIncludeDirectories>src\org\apache\hadoop\io\nativeio;%(AdditionalIncludeDirectories)</AdditionalIncludeDirectories>
     </ClCompile>
-    <ClCompile Include="src\org\apache\hadoop\io\erasurecode\erasure_code.c" Condition="'$(IsalEnabled)' == 'true'">
+    <ClCompile Include="src\org\apache\hadoop\io\erasurecode\isal_load.c" Condition="'$(IsalEnabled)' == 'true'">
       <AdditionalOptions>/D HADOOP_ISAL_LIBRARY=\"isa-l.dll\"</AdditionalOptions>
     </ClCompile>
-    <ClCompile Include="src\org\apache\hadoop\io\erasurecode\coder\erasure_code_native.c" Condition="'$(IsalEnabled)' == 'true'"/>
+    <ClCompile Include="src\org\apache\hadoop\io\erasurecode\erasure_code.c" Condition="'$(IsalEnabled)' == 'true'"/>
+    <ClCompile Include="src\org\apache\hadoop\io\erasurecode\gf_util.c" Condition="'$(IsalEnabled)' == 'true'"/>
+    <ClCompile Include="src\org\apache\hadoop\io\erasurecode\erasure_coder.c" Condition="'$(IsalEnabled)' == 'true'"/>
+    <ClCompile Include="src\org\apache\hadoop\io\erasurecode\dump.c" Condition="'$(IsalEnabled)' == 'true'"/>
+    <ClCompile Include="src\org\apache\hadoop\io\erasurecode\jni_erasure_code_native.c" Condition="'$(IsalEnabled)' == 'true'"/>
+    <ClCompile Include="src\org\apache\hadoop\io\erasurecode\jni_common.c" Condition="'$(IsalEnabled)' == 'true'"/>
+    <ClCompile Include="src\org\apache\hadoop\io\erasurecode\jni_rs_encoder.c" Condition="'$(IsalEnabled)' == 'true'"/>
+    <ClCompile Include="src\org\apache\hadoop\io\erasurecode\jni_rs_decoder.c" Condition="'$(IsalEnabled)' == 'true'"/>
   </ItemGroup>
   <ItemGroup>
     <ClInclude Include="..\src\org\apache\hadoop\util\crc32c_tables.h" />

http://git-wip-us.apache.org/repos/asf/hadoop/blob/658ee95f/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/erasurecode/coder/erasure_code_native.c
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/erasurecode/coder/erasure_code_native.c b/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/erasurecode/coder/erasure_code_native.c
deleted file mode 100644
index e84df9a..0000000
--- a/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/erasurecode/coder/erasure_code_native.c
+++ /dev/null
@@ -1,49 +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.
- */
-
-#include <stdio.h>
-#include <stdlib.h>
-#include <string.h>
-
-#include "org_apache_hadoop.h"
-#include "../include/erasure_code.h"
-#include "org_apache_hadoop_io_erasurecode_ErasureCodeNative.h"
-
-#ifdef UNIX
-#include "config.h"
-#endif
-
-JNIEXPORT void JNICALL
-Java_org_apache_hadoop_io_erasurecode_ErasureCodeNative_loadLibrary
-(JNIEnv *env, jclass myclass) {
-  char errMsg[1024];
-  load_erasurecode_lib(errMsg, sizeof(errMsg));
-  if (strlen(errMsg) > 0) {
-    THROW(env, "java/lang/UnsatisfiedLinkError", errMsg);
-  }
-}
-
-JNIEXPORT jstring JNICALL
-Java_org_apache_hadoop_io_erasurecode_ErasureCodeNative_getLibraryName
-(JNIEnv *env, jclass myclass) {
-  char* libName = get_library_name();
-  if (libName == NULL) {
-    libName = "Unavailable";
-  }
-  return (*env)->NewStringUTF(env, libName);
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/658ee95f/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/erasurecode/coder/org_apache_hadoop_io_erasurecode_ErasureCodeNative.h
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/erasurecode/coder/org_apache_hadoop_io_erasurecode_ErasureCodeNative.h b/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/erasurecode/coder/org_apache_hadoop_io_erasurecode_ErasureCodeNative.h
deleted file mode 100644
index d8ff3a0..0000000
--- a/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/erasurecode/coder/org_apache_hadoop_io_erasurecode_ErasureCodeNative.h
+++ /dev/null
@@ -1,29 +0,0 @@
-/* DO NOT EDIT THIS FILE - it is machine generated */
-#include <jni.h>
-/* Header for class org_apache_hadoop_io_erasurecode_ErasureCodeNative */
-
-#ifndef _Included_org_apache_hadoop_io_erasurecode_ErasureCodeNative
-#define _Included_org_apache_hadoop_io_erasurecode_ErasureCodeNative
-#ifdef __cplusplus
-extern "C" {
-#endif
-/*
- * Class:     org_apache_hadoop_io_erasurecode_ErasureCodeNative
- * Method:    loadLibrary
- * Signature: ()V
- */
-JNIEXPORT void JNICALL Java_org_apache_hadoop_io_erasurecode_ErasureCodeNative_loadLibrary
-  (JNIEnv *, jclass);
-
-/*
- * Class:     org_apache_hadoop_io_erasurecode_ErasureCodeNative
- * Method:    getLibraryName
- * Signature: ()Ljava/lang/String;
- */
-JNIEXPORT jstring JNICALL Java_org_apache_hadoop_io_erasurecode_ErasureCodeNative_getLibraryName
-  (JNIEnv *, jclass);
-
-#ifdef __cplusplus
-}
-#endif
-#endif

http://git-wip-us.apache.org/repos/asf/hadoop/blob/658ee95f/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/erasurecode/dump.c
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/erasurecode/dump.c b/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/erasurecode/dump.c
new file mode 100644
index 0000000..20bd189
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/erasurecode/dump.c
@@ -0,0 +1,100 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#include "erasure_code.h"
+#include "gf_util.h"
+#include "erasure_coder.h"
+
+#include <stdio.h>
+#include <stdlib.h>
+#include <string.h>
+
+void dump(unsigned char* buf, int len) {
+  int i;
+  for (i = 0; i < len;) {
+    printf(" %2x", 0xff & buf[i++]);
+    if (i % 32 == 0)
+      printf("\n");
+  }
+}
+
+void dumpMatrix(unsigned char** buf, int n1, int n2) {
+  int i, j;
+  for (i = 0; i < n1; i++) {
+    for (j = 0; j < n2; j++) {
+      printf(" %2x", buf[i][j]);
+    }
+    printf("\n");
+  }
+  printf("\n");
+}
+
+void dumpCodingMatrix(unsigned char* buf, int n1, int n2) {
+  int i, j;
+  for (i = 0; i < n1; i++) {
+    for (j = 0; j < n2; j++) {
+      printf(" %d", 0xff & buf[j + (i * n2)]);
+    }
+    printf("\n");
+  }
+  printf("\n");
+}
+
+void dumpEncoder(IsalEncoder* pCoder) {
+  int numDataUnits = pCoder->coder.numDataUnits;
+  int numParityUnits = pCoder->coder.numDataUnits;
+  int numAllUnits = pCoder->coder.numAllUnits;
+
+  printf("Encoding (numAlnumParityUnitslUnits = %d, numDataUnits = %d)\n",
+                                    numParityUnits, numDataUnits);
+
+  printf("\n\nEncodeMatrix:\n");
+  dumpCodingMatrix((unsigned char*) pCoder->encodeMatrix,
+                                           numDataUnits, numAllUnits);
+}
+
+void dumpDecoder(IsalDecoder* pCoder) {
+  int i, j;
+  int numDataUnits = pCoder->coder.numDataUnits;
+  int numAllUnits = pCoder->coder.numAllUnits;
+
+  printf("Recovering (numAllUnits = %d, numDataUnits = %d, numErased = %d)\n",
+                       numAllUnits, numDataUnits, pCoder->numErased);
+
+  printf(" - ErasedIndexes = ");
+  for (j = 0; j < pCoder->numErased; j++) {
+    printf(" %d", pCoder->erasedIndexes[j]);
+  }
+  printf("       - DecodeIndex = ");
+  for (i = 0; i < numDataUnits; i++) {
+    printf(" %d", pCoder->decodeIndex[i]);
+  }
+
+  printf("\n\nEncodeMatrix:\n");
+  dumpCodingMatrix((unsigned char*) pCoder->encodeMatrix,
+                                    numDataUnits, numAllUnits);
+
+  printf("InvertMatrix:\n");
+  dumpCodingMatrix((unsigned char*) pCoder->invertMatrix,
+                                   numDataUnits, numDataUnits);
+
+  printf("DecodeMatrix:\n");
+  dumpCodingMatrix((unsigned char*) pCoder->decodeMatrix,
+                                    numDataUnits, numAllUnits);
+}
+

http://git-wip-us.apache.org/repos/asf/hadoop/blob/658ee95f/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/erasurecode/dump.h
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/erasurecode/dump.h b/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/erasurecode/dump.h
new file mode 100644
index 0000000..f58d960
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/erasurecode/dump.h
@@ -0,0 +1,40 @@
+/**
+ * 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.
+ */
+
+/**
+ * Dump utilities for erasure coders.
+ */
+
+#ifndef _DUMP_H_
+#define _DUMP_H_
+
+#include <stdio.h>
+#include <stdlib.h>
+#include <string.h>
+
+void dumpEncoder(IsalEncoder* pCoder);
+
+void dumpDecoder(IsalDecoder* pCoder);
+
+void dump(unsigned char* buf, int len);
+
+void dumpMatrix(unsigned char** s, int k, int m);
+
+void dumpCodingMatrix(unsigned char* s, int n1, int n2);
+
+#endif //_DUMP_H_

http://git-wip-us.apache.org/repos/asf/hadoop/blob/658ee95f/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/erasurecode/erasure_code.c
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/erasurecode/erasure_code.c b/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/erasurecode/erasure_code.c
index a6c099a..1250688 100644
--- a/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/erasurecode/erasure_code.c
+++ b/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/erasurecode/erasure_code.c
@@ -20,252 +20,25 @@
 #include <stdlib.h>
 #include <string.h>
 
-#include "org_apache_hadoop.h"
-#include "../include/gf_util.h"
-#include "../include/erasure_code.h"
-
-#ifdef UNIX
-#include <sys/time.h>
-#include <sys/types.h>
-#include <sys/stat.h>
-#include <dlfcn.h>
-
-#include "config.h"
-#endif
-
-#ifdef WINDOWS
-#include <Windows.h>
-#endif
+#include "isal_load.h"
+#include "erasure_code.h"
 
 /**
  *  erasure_code.c
- *  Implementation erasure code utilities based on lib of erasure_code.so.
- *  Building of this codes won't rely on any ISA-L source codes, but running
- *  into this will rely on successfully loading of the dynamic library.
+ *  Implementation erasure code utilities based on ISA-L library.
  *
  */
 
-/**
- * The loaded library handle.
- */
-static void* libec = NULL;
-
-/**
- * A helper function to dlsym a 'symbol' from a given library-handle.
- */
-
-#ifdef UNIX
-
-static __attribute__ ((unused))
-void *my_dlsym(void *handle, const char *symbol) {
-  void *func_ptr = dlsym(handle, symbol);
-  return func_ptr;
-}
-
-/* A helper macro to dlsym the requisite dynamic symbol in NON-JNI env. */
-#define EC_LOAD_DYNAMIC_SYMBOL(func_ptr, handle, symbol) \
-  if ((func_ptr = my_dlsym(handle, symbol)) == NULL) { \
-    return "Failed to load symbol" symbol; \
-  }
-
-#endif
-
-#ifdef WINDOWS
-
-
-
-static FARPROC WINAPI my_dlsym(HMODULE handle, LPCSTR symbol) {
-  FARPROC func_ptr = GetProcAddress(handle, symbol);
-  return func_ptr;
-}
-
-/* A helper macro to dlsym the requisite dynamic symbol in NON-JNI env. */
-#define EC_LOAD_DYNAMIC_SYMBOL(func_type, func_ptr, handle, symbol) \
-  if ((func_ptr = (func_type)my_dlsym(handle, symbol)) == NULL) { \
-    return "Failed to load symbol" symbol; \
-  }
-
-#endif
-
-
-#ifdef UNIX
-// For gf_util.h
-static unsigned char (*d_gf_mul)(unsigned char, unsigned char);
-static unsigned char (*d_gf_inv)(unsigned char);
-static void (*d_gf_gen_rs_matrix)(unsigned char *, int, int);
-static void (*d_gf_gen_cauchy_matrix)(unsigned char *, int, int);
-static int (*d_gf_invert_matrix)(unsigned char *, unsigned char *, const int);
-static int (*d_gf_vect_mul)(int, unsigned char *, void *, void *);
-
-// For erasure_code.h
-static void (*d_ec_init_tables)(int, int, unsigned char*, unsigned char*);
-static void (*d_ec_encode_data)(int, int, int, unsigned char*,
-                                          unsigned char**, unsigned char**);
-static void (*d_ec_encode_data_update)(int, int, int, int, unsigned char*,
-                                             unsigned char*, unsigned char**);
-#endif
-
-#ifdef WINDOWS
-// For erasure_code.h
-typedef unsigned char (__cdecl *__d_gf_mul)(unsigned char, unsigned char);
-static __d_gf_mul d_gf_mul;
-typedef unsigned char (__cdecl *__d_gf_inv)(unsigned char);
-static __d_gf_inv d_gf_inv;
-typedef void (__cdecl *__d_gf_gen_rs_matrix)(unsigned char *, int, int);
-static __d_gf_gen_rs_matrix d_gf_gen_rs_matrix;
-typedef void (__cdecl *__d_gf_gen_cauchy_matrix)(unsigned char *, int, int);
-static __d_gf_gen_cauchy_matrix d_gf_gen_cauchy_matrix;
-typedef int (__cdecl *__d_gf_invert_matrix)(unsigned char *,
-                                                   unsigned char *, const int);
-static __d_gf_invert_matrix d_gf_invert_matrix;
-typedef int (__cdecl *__d_gf_vect_mul)(int, unsigned char *, void *, void *);
-static __d_gf_vect_mul d_gf_vect_mul;
-
-// For erasure_code.h
-typedef void (__cdecl *__d_ec_init_tables)(int, int,
-                                                unsigned char*, unsigned char*);
-static __d_ec_init_tables d_ec_init_tables;
-typedef void (__cdecl *__d_ec_encode_data)(int, int, int, unsigned char*,
-                                             unsigned char**, unsigned char**);
-static __d_ec_encode_data d_ec_encode_data;
-typedef void (__cdecl *__d_ec_encode_data_update)(int, int, int, int, unsigned char*,
-                                             unsigned char*, unsigned char**);
-static __d_ec_encode_data_update d_ec_encode_data_update;
-#endif
-
-static const char* load_functions(void* libec) {
-#ifdef UNIX
-  EC_LOAD_DYNAMIC_SYMBOL(d_gf_mul, libec, "gf_mul");
-  EC_LOAD_DYNAMIC_SYMBOL(d_gf_inv, libec, "gf_inv");
-  EC_LOAD_DYNAMIC_SYMBOL(d_gf_gen_rs_matrix, libec, "gf_gen_rs_matrix");
-  EC_LOAD_DYNAMIC_SYMBOL(d_gf_gen_cauchy_matrix, libec, "gf_gen_cauchy1_matrix");
-  EC_LOAD_DYNAMIC_SYMBOL(d_gf_invert_matrix, libec, "gf_invert_matrix");
-  EC_LOAD_DYNAMIC_SYMBOL(d_gf_vect_mul, libec, "gf_vect_mul");
-
-  EC_LOAD_DYNAMIC_SYMBOL(d_ec_init_tables, libec, "ec_init_tables");
-  EC_LOAD_DYNAMIC_SYMBOL(d_ec_encode_data, libec, "ec_encode_data");
-  EC_LOAD_DYNAMIC_SYMBOL(d_ec_encode_data_update, libec, "ec_encode_data_update");
-#endif
-
-#ifdef WINDOWS
-  EC_LOAD_DYNAMIC_SYMBOL(__d_gf_mul, d_gf_mul, libec, "gf_mul");
-  EC_LOAD_DYNAMIC_SYMBOL(__d_gf_inv, d_gf_inv, libec, "gf_inv");
-  EC_LOAD_DYNAMIC_SYMBOL(__d_gf_gen_rs_matrix, d_gf_gen_rs_matrix, libec, "gf_gen_rs_matrix");
-  EC_LOAD_DYNAMIC_SYMBOL(__d_gf_gen_cauchy_matrix, d_gf_gen_cauchy_matrix, libec, "gf_gen_cauchy1_matrix");
-  EC_LOAD_DYNAMIC_SYMBOL(__d_gf_invert_matrix, d_gf_invert_matrix, libec, "gf_invert_matrix");
-  EC_LOAD_DYNAMIC_SYMBOL(__d_gf_vect_mul, d_gf_vect_mul, libec, "gf_vect_mul");
-
-  EC_LOAD_DYNAMIC_SYMBOL(__d_ec_init_tables, d_ec_init_tables, libec, "ec_init_tables");
-  EC_LOAD_DYNAMIC_SYMBOL(__d_ec_encode_data, d_ec_encode_data, libec, "ec_encode_data");
-  EC_LOAD_DYNAMIC_SYMBOL(__d_ec_encode_data_update, d_ec_encode_data_update, libec, "ec_encode_data_update");
-#endif
-
-  return NULL;
-}
-
-void load_erasurecode_lib(char* err, size_t err_len) {
-  const char* errMsg;
-
-  err[0] = '\0';
-
-  if (libec != NULL) {
-    return;
-  }
-
-  // Load Intel ISA-L
-  #ifdef UNIX
-  libec = dlopen(HADOOP_ISAL_LIBRARY, RTLD_LAZY | RTLD_GLOBAL);
-  if (libec == NULL) {
-    snprintf(err, err_len, "Failed to load %s (%s)",
-                             HADOOP_ISAL_LIBRARY, dlerror());
-    return;
-  }
-  // Clear any existing error
-  dlerror();
-  #endif
-
-  #ifdef WINDOWS
-  libec = LoadLibrary(HADOOP_ISAL_LIBRARY);
-  if (libec == NULL) {
-    snprintf(err, err_len, "Failed to load %s", HADOOP_ISAL_LIBRARY);
-    return;
-  }
-  #endif
-
-  errMsg = load_functions(libec);
-  if (errMsg != NULL) {
-    snprintf(err, err_len, "Loading functions from ISA-L failed: %s", errMsg);
-  }
-}
-
-int build_support_erasurecode() {
-#ifdef HADOOP_ISAL_LIBRARY
-  return 1;
-#else
-  return 0;
-#endif
-}
-
-const char* get_library_name() {
-#ifdef UNIX
-  Dl_info dl_info;
-
-  if (d_ec_encode_data == NULL) {
-    return HADOOP_ISAL_LIBRARY;
-  }
-
-  if(dladdr(d_ec_encode_data, &dl_info)) {
-    return dl_info.dli_fname;
-  }
-#else
-  LPTSTR filename = NULL;
-
-  if (libec == NULL) {
-    return HADOOP_ISAL_LIBRARY;
-  }
-
-  if (GetModuleFileName(libec, filename, 256) > 0) {
-    return filename;
-  }
-#endif
-
-  return NULL;
-}
-
-unsigned char h_gf_mul(unsigned char a, unsigned char b) {
-  return d_gf_mul(a, b);
-}
-
-unsigned char h_gf_inv(unsigned char a) {
-  return d_gf_inv(a);
-}
-
-void h_gf_gen_rs_matrix(unsigned char *a, int m, int k) {
-  d_gf_gen_rs_matrix(a, m, k);
-}
-
-void h_gf_gen_cauchy_matrix(unsigned char *a, int m, int k) {
-  d_gf_gen_cauchy_matrix(a, m, k);
-}
-
-int h_gf_invert_matrix(unsigned char *in, unsigned char *out, const int n) {
-  return d_gf_invert_matrix(in, out, n);
-}
-
-int h_gf_vect_mul(int len, unsigned char *gftbl, void *src, void *dest) {
-  return d_gf_vect_mul(len, gftbl, src, dest);
-}
-
 void h_ec_init_tables(int k, int rows, unsigned char* a, unsigned char* gftbls) {
-  d_ec_init_tables(k, rows, a, gftbls);
+  isaLoader->ec_init_tables(k, rows, a, gftbls);
 }
 
 void h_ec_encode_data(int len, int k, int rows, unsigned char *gftbls,
     unsigned char **data, unsigned char **coding) {
-  d_ec_encode_data(len, k, rows, gftbls, data, coding);
+  isaLoader->ec_encode_data(len, k, rows, gftbls, data, coding);
 }
 
 void h_ec_encode_data_update(int len, int k, int rows, int vec_i,
          unsigned char *gftbls, unsigned char *data, unsigned char **coding) {
-  d_ec_encode_data_update(len, k, rows, vec_i, gftbls, data, coding);
+  isaLoader->ec_encode_data_update(len, k, rows, vec_i, gftbls, data, coding);
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/658ee95f/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/erasurecode/erasure_code.h
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/erasurecode/erasure_code.h b/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/erasurecode/erasure_code.h
new file mode 100644
index 0000000..a6cfd58
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/erasurecode/erasure_code.h
@@ -0,0 +1,107 @@
+/*
+ *  Licensed to the Apache Software Foundation (ASF) under one or more
+ *  contributor license agreements.  See the NOTICE file distributed with
+ *  this work for additional information regarding copyright ownership.
+ *  The ASF licenses this file to You under the Apache License, Version 2.0
+ *  (the "License"); you may not use this file except in compliance with
+ *  the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+#ifndef _ERASURE_CODE_H_
+#define _ERASURE_CODE_H_
+
+#include <stddef.h>
+
+/**
+ *  Interface to functions supporting erasure code encode and decode.
+ *
+ *  This file defines the interface to optimized functions used in erasure
+ *  codes.  Encode and decode of erasures in GF(2^8) are made by calculating the
+ *  dot product of the symbols (bytes in GF(2^8)) across a set of buffers and a
+ *  set of coefficients.  Values for the coefficients are determined by the type
+ *  of erasure code.  Using a general dot product means that any sequence of
+ *  coefficients may be used including erasure codes based on random
+ *  coefficients.
+ *  Multiple versions of dot product are supplied to calculate 1-6 output
+ *  vectors in one pass.
+ *  Base GF multiply and divide functions can be sped up by defining
+ *  GF_LARGE_TABLES at the expense of memory size.
+ *
+ */
+
+/**
+ * Initialize tables for fast Erasure Code encode and decode.
+ *
+ * Generates the expanded tables needed for fast encode or decode for erasure
+ * codes on blocks of data.  32bytes is generated for each input coefficient.
+ *
+ * @param k      The number of vector sources or rows in the generator matrix
+ *               for coding.
+ * @param rows   The number of output vectors to concurrently encode/decode.
+ * @param a      Pointer to sets of arrays of input coefficients used to encode
+ *               or decode data.
+ * @param gftbls Pointer to start of space for concatenated output tables
+ *               generated from input coefficients.  Must be of size 32*k*rows.
+ * @returns none
+ */
+void h_ec_init_tables(int k, int rows, unsigned char* a, unsigned char* gftbls);
+
+/**
+ * Generate or decode erasure codes on blocks of data, runs appropriate version.
+ *
+ * Given a list of source data blocks, generate one or multiple blocks of
+ * encoded data as specified by a matrix of GF(2^8) coefficients. When given a
+ * suitable set of coefficients, this function will perform the fast generation
+ * or decoding of Reed-Solomon type erasure codes.
+ *
+ * This function determines what instruction sets are enabled and
+ * selects the appropriate version at runtime.
+ *
+ * @param len    Length of each block of data (vector) of source or dest data.
+ * @param k      The number of vector sources or rows in the generator matrix
+ *        for coding.
+ * @param rows   The number of output vectors to concurrently encode/decode.
+ * @param gftbls Pointer to array of input tables generated from coding
+ *        coefficients in ec_init_tables(). Must be of size 32*k*rows
+ * @param data   Array of pointers to source input buffers.
+ * @param coding Array of pointers to coded output buffers.
+ * @returns none
+ */
+void h_ec_encode_data(int len, int k, int rows, unsigned char *gftbls,
+                                 unsigned char **data, unsigned char **coding);
+
+/**
+ * @brief Generate update for encode or decode of erasure codes from single
+ *        source, runs appropriate version.
+ *
+ * Given one source data block, update one or multiple blocks of encoded data as
+ * specified by a matrix of GF(2^8) coefficients. When given a suitable set of
+ * coefficients, this function will perform the fast generation or decoding of
+ * Reed-Solomon type erasure codes from one input source at a time.
+ *
+ * This function determines what instruction sets are enabled and selects the
+ * appropriate version at runtime.
+ *
+ * @param len    Length of each block of data (vector) of source or dest data.
+ * @param k      The number of vector sources or rows in the generator matrix
+ *               for coding.
+ * @param rows   The number of output vectors to concurrently encode/decode.
+ * @param vec_i  The vector index corresponding to the single input source.
+ * @param gftbls Pointer to array of input tables generated from coding
+ *               coefficients in ec_init_tables(). Must be of size 32*k*rows
+ * @param data   Pointer to single input source used to update output parity.
+ * @param coding Array of pointers to coded output buffers.
+ * @returns none
+ */
+void h_ec_encode_data_update(int len, int k, int rows, int vec_i,
+           unsigned char *gftbls, unsigned char *data, unsigned char **coding);
+
+#endif //_ERASURE_CODE_H_

http://git-wip-us.apache.org/repos/asf/hadoop/blob/658ee95f/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/erasurecode/erasure_coder.c
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/erasurecode/erasure_coder.c b/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/erasurecode/erasure_coder.c
new file mode 100644
index 0000000..b3479bb
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/erasurecode/erasure_coder.c
@@ -0,0 +1,229 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#include "erasure_code.h"
+#include "gf_util.h"
+#include "erasure_coder.h"
+
+#include <stdio.h>
+#include <stdlib.h>
+#include <string.h>
+
+void initCoder(IsalCoder* pCoder, int numDataUnits, int numParityUnits) {
+  pCoder->verbose = 0;
+  pCoder->numParityUnits = numParityUnits;
+  pCoder->numDataUnits = numDataUnits;
+  pCoder->numAllUnits = numDataUnits + numParityUnits;
+}
+
+// 0 not to verbose, 1 to verbose
+void allowVerbose(IsalCoder* pCoder, int flag) {
+  pCoder->verbose = flag;
+}
+
+static void initEncodeMatrix(int numDataUnits, int numParityUnits,
+                                                unsigned char* encodeMatrix) {
+  // Generate encode matrix, always invertible
+  h_gf_gen_cauchy_matrix(encodeMatrix,
+                          numDataUnits + numParityUnits, numDataUnits);
+}
+
+void initEncoder(IsalEncoder* pCoder, int numDataUnits,
+                            int numParityUnits) {
+  initCoder(&pCoder->coder, numDataUnits, numParityUnits);
+
+  initEncodeMatrix(numDataUnits, numParityUnits, pCoder->encodeMatrix);
+
+  // Generate gftbls from encode matrix
+  h_ec_init_tables(numDataUnits, numParityUnits,
+               &pCoder->encodeMatrix[numDataUnits * numDataUnits],
+               pCoder->gftbls);
+
+  if (pCoder->coder.verbose > 0) {
+    dumpEncoder(pCoder);
+  }
+}
+
+void initDecoder(IsalDecoder* pCoder, int numDataUnits,
+                                  int numParityUnits) {
+  initCoder(&pCoder->coder, numDataUnits, numParityUnits);
+
+  initEncodeMatrix(numDataUnits, numParityUnits, pCoder->encodeMatrix);
+}
+
+int encode(IsalEncoder* pCoder, unsigned char** dataUnits,
+    unsigned char** parityUnits, int chunkSize) {
+  int numDataUnits = pCoder->coder.numDataUnits;
+  int numParityUnits = pCoder->coder.numParityUnits;
+  int i;
+
+  for (i = 0; i < numParityUnits; i++) {
+    memset(parityUnits[i], 0, chunkSize);
+  }
+
+  h_ec_encode_data(chunkSize, numDataUnits, numParityUnits,
+                         pCoder->gftbls, dataUnits, parityUnits);
+
+  return 0;
+}
+
+// Return 1 when diff, 0 otherwise
+static int compare(int* arr1, int len1, int* arr2, int len2) {
+  int i;
+
+  if (len1 == len2) {
+    for (i = 0; i < len1; i++) {
+      if (arr1[i] != arr2[i]) {
+        return 1;
+      }
+    }
+    return 0;
+  }
+
+  return 1;
+}
+
+static int processErasures(IsalDecoder* pCoder, unsigned char** inputs,
+                                    int* erasedIndexes, int numErased) {
+  int i, r, ret, index;
+  int numDataUnits = pCoder->coder.numDataUnits;
+  int isChanged = 0;
+
+  for (i = 0, r = 0; i < numDataUnits; i++, r++) {
+    while (inputs[r] == NULL) {
+      r++;
+    }
+
+    if (pCoder->decodeIndex[i] != r) {
+      pCoder->decodeIndex[i] = r;
+      isChanged = 1;
+    }
+  }
+
+  for (i = 0; i < numDataUnits; i++) {
+    pCoder->realInputs[i] = inputs[pCoder->decodeIndex[i]];
+  }
+
+  if (isChanged == 0 &&
+          compare(pCoder->erasedIndexes, pCoder->numErased,
+                           erasedIndexes, numErased) == 0) {
+    return 0; // Optimization, nothing to do
+  }
+
+  clearDecoder(pCoder);
+
+  for (i = 0; i < numErased; i++) {
+    index = erasedIndexes[i];
+    pCoder->erasedIndexes[i] = index;
+    pCoder->erasureFlags[index] = 1;
+    if (index < numDataUnits) {
+      pCoder->numErasedDataUnits++;
+    }
+  }
+
+  pCoder->numErased = numErased;
+
+  ret = generateDecodeMatrix(pCoder);
+  if (ret != 0) {
+    printf("Failed to generate decode matrix\n");
+    return -1;
+  }
+
+  h_ec_init_tables(numDataUnits, pCoder->numErased,
+                      pCoder->decodeMatrix, pCoder->gftbls);
+
+  if (pCoder->coder.verbose > 0) {
+    dumpDecoder(pCoder);
+  }
+
+  return 0;
+}
+
+int decode(IsalDecoder* pCoder, unsigned char** inputs,
+                  int* erasedIndexes, int numErased,
+                   unsigned char** outputs, int chunkSize) {
+  int numDataUnits = pCoder->coder.numDataUnits;
+  int i;
+
+  processErasures(pCoder, inputs, erasedIndexes, numErased);
+
+  for (i = 0; i < numErased; i++) {
+    memset(outputs[i], 0, chunkSize);
+  }
+
+  h_ec_encode_data(chunkSize, numDataUnits, pCoder->numErased,
+      pCoder->gftbls, pCoder->realInputs, outputs);
+
+  return 0;
+}
+
+// Clear variables used per decode call
+void clearDecoder(IsalDecoder* decoder) {
+  decoder->numErasedDataUnits = 0;
+  decoder->numErased = 0;
+  memset(decoder->gftbls, 0, sizeof(decoder->gftbls));
+  memset(decoder->decodeMatrix, 0, sizeof(decoder->decodeMatrix));
+  memset(decoder->tmpMatrix, 0, sizeof(decoder->tmpMatrix));
+  memset(decoder->invertMatrix, 0, sizeof(decoder->invertMatrix));
+  memset(decoder->erasureFlags, 0, sizeof(decoder->erasureFlags));
+  memset(decoder->erasedIndexes, 0, sizeof(decoder->erasedIndexes));
+}
+
+// Generate decode matrix from encode matrix
+int generateDecodeMatrix(IsalDecoder* pCoder) {
+  int i, j, r, p;
+  unsigned char s;
+  int numDataUnits;
+
+  numDataUnits = pCoder->coder.numDataUnits;
+
+  // Construct matrix b by removing error rows
+  for (i = 0; i < numDataUnits; i++) {
+    r = pCoder->decodeIndex[i];
+    for (j = 0; j < numDataUnits; j++) {
+      pCoder->tmpMatrix[numDataUnits * i + j] =
+                pCoder->encodeMatrix[numDataUnits * r + j];
+    }
+  }
+
+  h_gf_invert_matrix(pCoder->tmpMatrix,
+                                pCoder->invertMatrix, numDataUnits);
+
+  for (i = 0; i < pCoder->numErasedDataUnits; i++) {
+    for (j = 0; j < numDataUnits; j++) {
+      pCoder->decodeMatrix[numDataUnits * i + j] =
+                      pCoder->invertMatrix[numDataUnits *
+                      pCoder->erasedIndexes[i] + j];
+    }
+  }
+
+  for (p = pCoder->numErasedDataUnits; p < pCoder->numErased; p++) {
+    for (i = 0; i < numDataUnits; i++) {
+      s = 0;
+      for (j = 0; j < numDataUnits; j++) {
+        s ^= h_gf_mul(pCoder->invertMatrix[j * numDataUnits + i],
+          pCoder->encodeMatrix[numDataUnits *
+                                        pCoder->erasedIndexes[p] + j]);
+      }
+
+      pCoder->decodeMatrix[numDataUnits * p + i] = s;
+    }
+  }
+
+  return 0;
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/658ee95f/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/erasurecode/erasure_coder.h
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/erasurecode/erasure_coder.h b/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/erasurecode/erasure_coder.h
new file mode 100644
index 0000000..8f5bf8a
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/erasurecode/erasure_coder.h
@@ -0,0 +1,88 @@
+/**
+ * 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.
+ */
+
+/**
+ * This is a sample program illustrating how to use the Intel ISA-L library.
+ * Note it's adapted from erasure_code_test.c test program, but trying to use
+ * variable names and styles we're more familiar with already similar to Java
+ * coders.
+ */
+
+#ifndef _ERASURE_CODER_H_
+#define _ERASURE_CODER_H_
+
+#include <stdio.h>
+#include <stdlib.h>
+#include <string.h>
+
+#define MMAX 14
+#define KMAX 10
+
+typedef struct _IsalCoder {
+  int verbose;
+  int numParityUnits;
+  int numDataUnits;
+  int numAllUnits;
+} IsalCoder;
+
+typedef struct _IsalEncoder {
+  IsalCoder coder;
+
+  unsigned char gftbls[MMAX * KMAX * 32];
+
+  unsigned char encodeMatrix[MMAX * KMAX];
+} IsalEncoder;
+
+typedef struct _IsalDecoder {
+  IsalCoder coder;
+
+  unsigned char encodeMatrix[MMAX * KMAX];
+
+  // Below are per decode call
+  unsigned char gftbls[MMAX * KMAX * 32];
+  unsigned int decodeIndex[MMAX];
+  unsigned char tmpMatrix[MMAX * KMAX];
+  unsigned char invertMatrix[MMAX * KMAX];
+  unsigned char decodeMatrix[MMAX * KMAX];
+  unsigned char erasureFlags[MMAX];
+  int erasedIndexes[MMAX];
+  int numErased;
+  int numErasedDataUnits;
+  unsigned char* realInputs[MMAX];
+} IsalDecoder;
+
+void initCoder(IsalCoder* pCoder, int numDataUnits, int numParityUnits);
+
+void allowVerbose(IsalCoder* pCoder, int flag);
+
+void initEncoder(IsalEncoder* encoder, int numDataUnits, int numParityUnits);
+
+void initDecoder(IsalDecoder* decoder, int numDataUnits, int numParityUnits);
+
+void clearDecoder(IsalDecoder* decoder);
+
+int encode(IsalEncoder* encoder, unsigned char** dataUnits,
+    unsigned char** parityUnits, int chunkSize);
+
+int decode(IsalDecoder* decoder, unsigned char** allUnits,
+    int* erasedIndexes, int numErased,
+    unsigned char** recoveredUnits, int chunkSize);
+
+int generateDecodeMatrix(IsalDecoder* pCoder);
+
+#endif //_ERASURE_CODER_H_

http://git-wip-us.apache.org/repos/asf/hadoop/blob/658ee95f/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/erasurecode/gf_util.c
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/erasurecode/gf_util.c b/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/erasurecode/gf_util.c
new file mode 100644
index 0000000..804dcec
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/erasurecode/gf_util.c
@@ -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.
+ */
+
+#include <errno.h>
+#include <stdio.h>
+#include <stdlib.h>
+#include <string.h>
+
+#include "isal_load.h"
+#include "gf_util.h"
+
+/**
+ *  gf_util.c
+ *  Implementation GF utilities based on ISA-L library.
+ *
+ */
+
+unsigned char h_gf_mul(unsigned char a, unsigned char b) {
+  return isaLoader->gf_mul(a, b);
+}
+
+unsigned char h_gf_inv(unsigned char a) {
+  return isaLoader->gf_inv(a);
+}
+
+void h_gf_gen_rs_matrix(unsigned char *a, int m, int k) {
+  isaLoader->gf_gen_rs_matrix(a, m, k);
+}
+
+void h_gf_gen_cauchy_matrix(unsigned char *a, int m, int k) {
+  isaLoader->gf_gen_cauchy_matrix(a, m, k);
+}
+
+int h_gf_invert_matrix(unsigned char *in, unsigned char *out, const int n) {
+  return isaLoader->gf_invert_matrix(in, out, n);
+}
+
+int h_gf_vect_mul(int len, unsigned char *gftbl, void *src, void *dest) {
+  return isaLoader->gf_vect_mul(len, gftbl, src, dest);
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/658ee95f/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/erasurecode/gf_util.h
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/erasurecode/gf_util.h b/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/erasurecode/gf_util.h
new file mode 100644
index 0000000..2be8328
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/erasurecode/gf_util.h
@@ -0,0 +1,111 @@
+/*
+ *  Licensed to the Apache Software Foundation (ASF) under one or more
+ *  contributor license agreements.  See the NOTICE file distributed with
+ *  this work for additional information regarding copyright ownership.
+ *  The ASF licenses this file to You under the Apache License, Version 2.0
+ *  (the "License"); you may not use this file except in compliance with
+ *  the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+#ifndef _GF_UTIL_H
+#define _GF_UTIL_H
+
+/**
+ *  gf_util.h
+ *  Interface to functions for vector (block) multiplication in GF(2^8).
+ *
+ *  This file defines the interface to routines used in fast RAID rebuild and
+ *  erasure codes.
+ */
+
+
+/**
+ * Single element GF(2^8) multiply.
+ *
+ * @param a  Multiplicand a
+ * @param b  Multiplicand b
+ * @returns  Product of a and b in GF(2^8)
+ */
+unsigned char h_gf_mul(unsigned char a, unsigned char b);
+
+/**
+ * Single element GF(2^8) inverse.
+ *
+ * @param a  Input element
+ * @returns  Field element b such that a x b = {1}
+ */
+unsigned char h_gf_inv(unsigned char a);
+
+/**
+ * Generate a matrix of coefficients to be used for encoding.
+ *
+ * Vandermonde matrix example of encoding coefficients where high portion of
+ * matrix is identity matrix I and lower portion is constructed as 2^{i*(j-k+1)}
+ * i:{0,k-1} j:{k,m-1}. Commonly used method for choosing coefficients in
+ * erasure encoding but does not guarantee invertable for every sub matrix.  For
+ * large k it is possible to find cases where the decode matrix chosen from
+ * sources and parity not in erasure are not invertable. Users may want to
+ * adjust for k > 5.
+ *
+ * @param a  [mxk] array to hold coefficients
+ * @param m  number of rows in matrix corresponding to srcs + parity.
+ * @param k  number of columns in matrix corresponding to srcs.
+ * @returns  none
+ */
+void h_gf_gen_rs_matrix(unsigned char *a, int m, int k);
+
+/**
+ * Generate a Cauchy matrix of coefficients to be used for encoding.
+ *
+ * Cauchy matrix example of encoding coefficients where high portion of matrix
+ * is identity matrix I and lower portion is constructed as 1/(i + j) | i != j,
+ * i:{0,k-1} j:{k,m-1}.  Any sub-matrix of a Cauchy matrix should be invertable.
+ *
+ * @param a  [mxk] array to hold coefficients
+ * @param m  number of rows in matrix corresponding to srcs + parity.
+ * @param k  number of columns in matrix corresponding to srcs.
+ * @returns  none
+ */
+void h_gf_gen_cauchy_matrix(unsigned char *a, int m, int k);
+
+/**
+ * Invert a matrix in GF(2^8)
+ *
+ * @param in  input matrix
+ * @param out output matrix such that [in] x [out] = [I] - identity matrix
+ * @param n   size of matrix [nxn]
+ * @returns 0 successful, other fail on singular input matrix
+ */
+int h_gf_invert_matrix(unsigned char *in, unsigned char *out, const int n);
+
+/**
+ * GF(2^8) vector multiply by constant, runs appropriate version.
+ *
+ * Does a GF(2^8) vector multiply b = Ca where a and b are arrays and C
+ * is a single field element in GF(2^8). Can be used for RAID6 rebuild
+ * and partial write functions. Function requires pre-calculation of a
+ * 32-element constant array based on constant C. gftbl(C) = {C{00},
+ * C{01}, C{02}, ... , C{0f} }, {C{00}, C{10}, C{20}, ... , C{f0} }.
+ * Len and src must be aligned to 32B.
+ *
+ * This function determines what instruction sets are enabled
+ * and selects the appropriate version at runtime.
+ *
+ * @param len   Length of vector in bytes. Must be aligned to 32B.
+ * @param gftbl Pointer to 32-byte array of pre-calculated constants based on C.
+ * @param src   Pointer to src data array. Must be aligned to 32B.
+ * @param dest  Pointer to destination data array. Must be aligned to 32B.
+ * @returns 0 pass, other fail
+ */
+int h_gf_vect_mul(int len, unsigned char *gftbl, void *src, void *dest);
+
+
+#endif //_GF_UTIL_H

http://git-wip-us.apache.org/repos/asf/hadoop/blob/658ee95f/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/erasurecode/include/erasure_code.h
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/erasurecode/include/erasure_code.h b/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/erasurecode/include/erasure_code.h
deleted file mode 100644
index 123085e..0000000
--- a/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/erasurecode/include/erasure_code.h
+++ /dev/null
@@ -1,125 +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.
- */
-
-#ifndef _ERASURE_CODE_H_
-#define _ERASURE_CODE_H_
-
-#include <stddef.h>
-
-/**
- *  Interface to functions supporting erasure code encode and decode.
- *
- *  This file defines the interface to optimized functions used in erasure
- *  codes.  Encode and decode of erasures in GF(2^8) are made by calculating the
- *  dot product of the symbols (bytes in GF(2^8)) across a set of buffers and a
- *  set of coefficients.  Values for the coefficients are determined by the type
- *  of erasure code.  Using a general dot product means that any sequence of
- *  coefficients may be used including erasure codes based on random
- *  coefficients.
- *  Multiple versions of dot product are supplied to calculate 1-6 output
- *  vectors in one pass.
- *  Base GF multiply and divide functions can be sped up by defining
- *  GF_LARGE_TABLES at the expense of memory size.
- *
- */
-
-/**
- * Return 0 if not support, 1 otherwise.
- */
-int build_support_erasurecode();
-
-/**
- * Get the library name possibly of full path.
- */
-const char* get_library_name();
-
-/**
- * Initialize and load erasure code library, returning error message if any.
- *
- * @param err     The err message buffer.
- * @param err_len The length of the message buffer.
- */
-void load_erasurecode_lib(char* err, size_t err_len);
-
-/**
- * Initialize tables for fast Erasure Code encode and decode.
- *
- * Generates the expanded tables needed for fast encode or decode for erasure
- * codes on blocks of data.  32bytes is generated for each input coefficient.
- *
- * @param k      The number of vector sources or rows in the generator matrix
- *               for coding.
- * @param rows   The number of output vectors to concurrently encode/decode.
- * @param a      Pointer to sets of arrays of input coefficients used to encode
- *               or decode data.
- * @param gftbls Pointer to start of space for concatenated output tables
- *               generated from input coefficients.  Must be of size 32*k*rows.
- * @returns none
- */
-void h_ec_init_tables(int k, int rows, unsigned char* a, unsigned char* gftbls);
-
-/**
- * Generate or decode erasure codes on blocks of data, runs appropriate version.
- *
- * Given a list of source data blocks, generate one or multiple blocks of
- * encoded data as specified by a matrix of GF(2^8) coefficients. When given a
- * suitable set of coefficients, this function will perform the fast generation
- * or decoding of Reed-Solomon type erasure codes.
- *
- * This function determines what instruction sets are enabled and
- * selects the appropriate version at runtime.
- *
- * @param len    Length of each block of data (vector) of source or dest data.
- * @param k      The number of vector sources or rows in the generator matrix
- *        for coding.
- * @param rows   The number of output vectors to concurrently encode/decode.
- * @param gftbls Pointer to array of input tables generated from coding
- *        coefficients in ec_init_tables(). Must be of size 32*k*rows
- * @param data   Array of pointers to source input buffers.
- * @param coding Array of pointers to coded output buffers.
- * @returns none
- */
-void h_ec_encode_data(int len, int k, int rows, unsigned char *gftbls,
-                                 unsigned char **data, unsigned char **coding);
-
-/**
- * @brief Generate update for encode or decode of erasure codes from single
- *        source, runs appropriate version.
- *
- * Given one source data block, update one or multiple blocks of encoded data as
- * specified by a matrix of GF(2^8) coefficients. When given a suitable set of
- * coefficients, this function will perform the fast generation or decoding of
- * Reed-Solomon type erasure codes from one input source at a time.
- *
- * This function determines what instruction sets are enabled and selects the
- * appropriate version at runtime.
- *
- * @param len    Length of each block of data (vector) of source or dest data.
- * @param k      The number of vector sources or rows in the generator matrix
- *               for coding.
- * @param rows   The number of output vectors to concurrently encode/decode.
- * @param vec_i  The vector index corresponding to the single input source.
- * @param gftbls Pointer to array of input tables generated from coding
- *               coefficients in ec_init_tables(). Must be of size 32*k*rows
- * @param data   Pointer to single input source used to update output parity.
- * @param coding Array of pointers to coded output buffers.
- * @returns none
- */
-void h_ec_encode_data_update(int len, int k, int rows, int vec_i,
-           unsigned char *gftbls, unsigned char *data, unsigned char **coding);
-
-#endif //_ERASURE_CODE_H_

http://git-wip-us.apache.org/repos/asf/hadoop/blob/658ee95f/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/erasurecode/include/gf_util.h
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/erasurecode/include/gf_util.h b/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/erasurecode/include/gf_util.h
deleted file mode 100644
index 2be8328..0000000
--- a/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/erasurecode/include/gf_util.h
+++ /dev/null
@@ -1,111 +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.
- */
-
-#ifndef _GF_UTIL_H
-#define _GF_UTIL_H
-
-/**
- *  gf_util.h
- *  Interface to functions for vector (block) multiplication in GF(2^8).
- *
- *  This file defines the interface to routines used in fast RAID rebuild and
- *  erasure codes.
- */
-
-
-/**
- * Single element GF(2^8) multiply.
- *
- * @param a  Multiplicand a
- * @param b  Multiplicand b
- * @returns  Product of a and b in GF(2^8)
- */
-unsigned char h_gf_mul(unsigned char a, unsigned char b);
-
-/**
- * Single element GF(2^8) inverse.
- *
- * @param a  Input element
- * @returns  Field element b such that a x b = {1}
- */
-unsigned char h_gf_inv(unsigned char a);
-
-/**
- * Generate a matrix of coefficients to be used for encoding.
- *
- * Vandermonde matrix example of encoding coefficients where high portion of
- * matrix is identity matrix I and lower portion is constructed as 2^{i*(j-k+1)}
- * i:{0,k-1} j:{k,m-1}. Commonly used method for choosing coefficients in
- * erasure encoding but does not guarantee invertable for every sub matrix.  For
- * large k it is possible to find cases where the decode matrix chosen from
- * sources and parity not in erasure are not invertable. Users may want to
- * adjust for k > 5.
- *
- * @param a  [mxk] array to hold coefficients
- * @param m  number of rows in matrix corresponding to srcs + parity.
- * @param k  number of columns in matrix corresponding to srcs.
- * @returns  none
- */
-void h_gf_gen_rs_matrix(unsigned char *a, int m, int k);
-
-/**
- * Generate a Cauchy matrix of coefficients to be used for encoding.
- *
- * Cauchy matrix example of encoding coefficients where high portion of matrix
- * is identity matrix I and lower portion is constructed as 1/(i + j) | i != j,
- * i:{0,k-1} j:{k,m-1}.  Any sub-matrix of a Cauchy matrix should be invertable.
- *
- * @param a  [mxk] array to hold coefficients
- * @param m  number of rows in matrix corresponding to srcs + parity.
- * @param k  number of columns in matrix corresponding to srcs.
- * @returns  none
- */
-void h_gf_gen_cauchy_matrix(unsigned char *a, int m, int k);
-
-/**
- * Invert a matrix in GF(2^8)
- *
- * @param in  input matrix
- * @param out output matrix such that [in] x [out] = [I] - identity matrix
- * @param n   size of matrix [nxn]
- * @returns 0 successful, other fail on singular input matrix
- */
-int h_gf_invert_matrix(unsigned char *in, unsigned char *out, const int n);
-
-/**
- * GF(2^8) vector multiply by constant, runs appropriate version.
- *
- * Does a GF(2^8) vector multiply b = Ca where a and b are arrays and C
- * is a single field element in GF(2^8). Can be used for RAID6 rebuild
- * and partial write functions. Function requires pre-calculation of a
- * 32-element constant array based on constant C. gftbl(C) = {C{00},
- * C{01}, C{02}, ... , C{0f} }, {C{00}, C{10}, C{20}, ... , C{f0} }.
- * Len and src must be aligned to 32B.
- *
- * This function determines what instruction sets are enabled
- * and selects the appropriate version at runtime.
- *
- * @param len   Length of vector in bytes. Must be aligned to 32B.
- * @param gftbl Pointer to 32-byte array of pre-calculated constants based on C.
- * @param src   Pointer to src data array. Must be aligned to 32B.
- * @param dest  Pointer to destination data array. Must be aligned to 32B.
- * @returns 0 pass, other fail
- */
-int h_gf_vect_mul(int len, unsigned char *gftbl, void *src, void *dest);
-
-
-#endif //_GF_UTIL_H

http://git-wip-us.apache.org/repos/asf/hadoop/blob/658ee95f/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/erasurecode/isal_load.c
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/erasurecode/isal_load.c b/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/erasurecode/isal_load.c
new file mode 100644
index 0000000..55e8efd
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/erasurecode/isal_load.c
@@ -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.
+ */
+
+#include <errno.h>
+#include <stdio.h>
+#include <stdlib.h>
+#include <string.h>
+
+#include "org_apache_hadoop.h"
+#include "isal_load.h"
+
+#ifdef UNIX
+#include <sys/time.h>
+#include <sys/types.h>
+#include <sys/stat.h>
+#include <dlfcn.h>
+
+#include "config.h"
+#endif
+
+#ifdef WINDOWS
+#include <Windows.h>
+#endif
+
+IsaLibLoader* isaLoader;
+
+/**
+ *  isal_load.c
+ *  Utility of loading the ISA-L library and the required functions.
+ *  Building of this codes won't rely on any ISA-L source codes, but running
+ *  into this will rely on successfully loading of the dynamic library.
+ *
+ */
+
+static const char* load_functions() {
+#ifdef UNIX
+  EC_LOAD_DYNAMIC_SYMBOL((isaLoader->gf_mul), "gf_mul");
+  EC_LOAD_DYNAMIC_SYMBOL((isaLoader->gf_inv), "gf_inv");
+  EC_LOAD_DYNAMIC_SYMBOL((isaLoader->gf_gen_rs_matrix), "gf_gen_rs_matrix");
+  EC_LOAD_DYNAMIC_SYMBOL((isaLoader->gf_gen_cauchy_matrix), "gf_gen_cauchy1_matrix");
+  EC_LOAD_DYNAMIC_SYMBOL((isaLoader->gf_invert_matrix), "gf_invert_matrix");
+  EC_LOAD_DYNAMIC_SYMBOL((isaLoader->gf_vect_mul), "gf_vect_mul");
+
+  EC_LOAD_DYNAMIC_SYMBOL((isaLoader->ec_init_tables), "ec_init_tables");
+  EC_LOAD_DYNAMIC_SYMBOL((isaLoader->ec_encode_data), "ec_encode_data");
+  EC_LOAD_DYNAMIC_SYMBOL((isaLoader->ec_encode_data_update), "ec_encode_data_update");
+#endif
+
+#ifdef WINDOWS
+  EC_LOAD_DYNAMIC_SYMBOL(__d_gf_mul, (isaLoader->gf_mul), "gf_mul");
+  EC_LOAD_DYNAMIC_SYMBOL(__d_gf_inv, (isaLoader->gf_inv), "gf_inv");
+  EC_LOAD_DYNAMIC_SYMBOL(__d_gf_gen_rs_matrix, (isaLoader->gf_gen_rs_matrix), "gf_gen_rs_matrix");
+  EC_LOAD_DYNAMIC_SYMBOL(__d_gf_gen_cauchy_matrix, (isaLoader->gf_gen_cauchy_matrix), "gf_gen_cauchy1_matrix");
+  EC_LOAD_DYNAMIC_SYMBOL(__d_gf_invert_matrix, (isaLoader->gf_invert_matrix), "gf_invert_matrix");
+  EC_LOAD_DYNAMIC_SYMBOL(__d_gf_vect_mul, (isaLoader->gf_vect_mul), "gf_vect_mul");
+
+  EC_LOAD_DYNAMIC_SYMBOL(__d_ec_init_tables, (isaLoader->ec_init_tables), "ec_init_tables");
+  EC_LOAD_DYNAMIC_SYMBOL(__d_ec_encode_data, (isaLoader->ec_encode_data), "ec_encode_data");
+  EC_LOAD_DYNAMIC_SYMBOL(__d_ec_encode_data_update, (isaLoader->ec_encode_data_update), "ec_encode_data_update");
+#endif
+
+  return NULL;
+}
+
+void load_erasurecode_lib(char* err, size_t err_len) {
+  const char* errMsg;
+
+  err[0] = '\0';
+
+  if (isaLoader != NULL) {
+    return;
+  }
+  isaLoader = calloc(1, sizeof(IsaLibLoader));
+  memset(isaLoader, 0, sizeof(IsaLibLoader));
+
+  // Load Intel ISA-L
+  #ifdef UNIX
+  isaLoader->libec = dlopen(HADOOP_ISAL_LIBRARY, RTLD_LAZY | RTLD_GLOBAL);
+  if (isaLoader->libec == NULL) {
+    snprintf(err, err_len, "Failed to load %s (%s)",
+                             HADOOP_ISAL_LIBRARY, dlerror());
+    return;
+  }
+  // Clear any existing error
+  dlerror();
+  #endif
+
+  #ifdef WINDOWS
+  isaLoader->libec = LoadLibrary(HADOOP_ISAL_LIBRARY);
+  if (isaLoader->libec == NULL) {
+    snprintf(err, err_len, "Failed to load %s", HADOOP_ISAL_LIBRARY);
+    return;
+  }
+  #endif
+
+  errMsg = load_functions(isaLoader->libec);
+  if (errMsg != NULL) {
+    snprintf(err, err_len, "Loading functions from ISA-L failed: %s", errMsg);
+  }
+}
+
+int build_support_erasurecode() {
+#ifdef HADOOP_ISAL_LIBRARY
+  return 1;
+#else
+  return 0;
+#endif
+}
+
+const char* get_library_name() {
+#ifdef UNIX
+  Dl_info dl_info;
+
+  if (isaLoader->ec_encode_data == NULL) {
+    return HADOOP_ISAL_LIBRARY;
+  }
+
+  if(dladdr(isaLoader->ec_encode_data, &dl_info)) {
+    return dl_info.dli_fname;
+  }
+#else
+  LPTSTR filename = NULL;
+
+  if (isaLoader->libec == NULL) {
+    return HADOOP_ISAL_LIBRARY;
+  }
+
+  if (GetModuleFileName(isaLoader->libec, filename, 256) > 0) {
+    return filename;
+  }
+#endif
+
+  return NULL;
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/658ee95f/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/erasurecode/isal_load.h
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/erasurecode/isal_load.h b/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/erasurecode/isal_load.h
new file mode 100644
index 0000000..635706d
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/erasurecode/isal_load.h
@@ -0,0 +1,149 @@
+/*
+ *  Licensed to the Apache Software Foundation (ASF) under one or more
+ *  contributor license agreements.  See the NOTICE file distributed with
+ *  this work for additional information regarding copyright ownership.
+ *  The ASF licenses this file to You under the Apache License, Version 2.0
+ *  (the "License"); you may not use this file except in compliance with
+ *  the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+#include <errno.h>
+#include <stdio.h>
+#include <stdlib.h>
+#include <string.h>
+
+#include "org_apache_hadoop.h"
+
+#ifdef UNIX
+#include <sys/time.h>
+#include <sys/types.h>
+#include <sys/stat.h>
+#include <dlfcn.h>
+
+#include "config.h"
+#endif
+
+#ifdef WINDOWS
+#include <Windows.h>
+#endif
+
+#ifndef _ISAL_LOAD_H_
+#define _ISAL_LOAD_H_
+
+
+#ifdef UNIX
+// For gf_util.h
+typedef unsigned char (*__d_gf_mul)(unsigned char, unsigned char);
+typedef unsigned char (*__d_gf_inv)(unsigned char);
+typedef void (*__d_gf_gen_rs_matrix)(unsigned char *, int, int);
+typedef void (*__d_gf_gen_cauchy_matrix)(unsigned char *, int, int);
+typedef int (*__d_gf_invert_matrix)(unsigned char *, unsigned char *, const int);
+typedef int (*__d_gf_vect_mul)(int, unsigned char *, void *, void *);
+
+// For erasure_code.h
+typedef void (*__d_ec_init_tables)(int, int, unsigned char*, unsigned char*);
+typedef void (*__d_ec_encode_data)(int, int, int, unsigned char*,
+                                          unsigned char**, unsigned char**);
+typedef void (*__d_ec_encode_data_update)(int, int, int, int, unsigned char*,
+                                             unsigned char*, unsigned char**);
+#endif
+
+#ifdef WINDOWS
+// For erasure_code.h
+typedef unsigned char (__cdecl *__d_gf_mul)(unsigned char, unsigned char);
+typedef unsigned char (__cdecl *__d_gf_inv)(unsigned char);
+typedef void (__cdecl *__d_gf_gen_rs_matrix)(unsigned char *, int, int);
+typedef void (__cdecl *__d_gf_gen_cauchy_matrix)(unsigned char *, int, int);
+typedef int (__cdecl *__d_gf_invert_matrix)(unsigned char *,
+                                                unsigned char *, const int);
+typedef int (__cdecl *__d_gf_vect_mul)(int, unsigned char *, void *, void *);
+
+// For erasure_code.h
+typedef void (__cdecl *__d_ec_init_tables)(int, int,
+                                                unsigned char*, unsigned char*);
+typedef void (__cdecl *__d_ec_encode_data)(int, int, int, unsigned char*,
+                                             unsigned char**, unsigned char**);
+typedef void (__cdecl *__d_ec_encode_data_update)(int, int, int, int, unsigned char*,
+                                             unsigned char*, unsigned char**);
+#endif
+
+typedef struct __IsaLibLoader {
+  // The loaded library handle
+  void* libec;
+
+  __d_gf_mul gf_mul;
+  __d_gf_inv gf_inv;
+  __d_gf_gen_rs_matrix gf_gen_rs_matrix;
+  __d_gf_gen_cauchy_matrix gf_gen_cauchy_matrix;
+  __d_gf_invert_matrix gf_invert_matrix;
+  __d_gf_vect_mul gf_vect_mul;
+  __d_ec_init_tables ec_init_tables;
+  __d_ec_encode_data ec_encode_data;
+  __d_ec_encode_data_update ec_encode_data_update;
+} IsaLibLoader;
+
+extern IsaLibLoader* isaLoader;
+
+/**
+ * A helper function to dlsym a 'symbol' from a given library-handle.
+ */
+
+#ifdef UNIX
+
+static __attribute__ ((unused))
+void *myDlsym(void *handle, const char *symbol) {
+  void *func_ptr = dlsym(handle, symbol);
+  return func_ptr;
+}
+
+/* A helper macro to dlsym the requisite dynamic symbol in NON-JNI env. */
+#define EC_LOAD_DYNAMIC_SYMBOL(func_ptr, symbol) \
+  if ((func_ptr = myDlsym(isaLoader->libec, symbol)) == NULL) { \
+    return "Failed to load symbol" symbol; \
+  }
+
+#endif
+
+#ifdef WINDOWS
+
+
+static FARPROC WINAPI myDlsym(HMODULE handle, LPCSTR symbol) {
+  FARPROC func_ptr = GetProcAddress(handle, symbol);
+  return func_ptr;
+}
+
+/* A helper macro to dlsym the requisite dynamic symbol in NON-JNI env. */
+#define EC_LOAD_DYNAMIC_SYMBOL(func_type, func_ptr, symbol) \
+  if ((func_ptr = (func_type)myDlsym(isaLoader->libec, symbol)) == NULL) { \
+    return "Failed to load symbol" symbol; \
+  }
+
+#endif
+
+/**
+ * Return 0 if not support, 1 otherwise.
+ */
+int build_support_erasurecode();
+
+/**
+ * Get the library name possibly of full path.
+ */
+const char* get_library_name();
+
+/**
+ * Initialize and load erasure code library, returning error message if any.
+ *
+ * @param err     The err message buffer.
+ * @param err_len The length of the message buffer.
+ */
+void load_erasurecode_lib(char* err, size_t err_len);
+
+#endif //_ISAL_LOAD_H_
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/658ee95f/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/erasurecode/jni_common.c
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/erasurecode/jni_common.c b/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/erasurecode/jni_common.c
new file mode 100644
index 0000000..8126e9a
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/erasurecode/jni_common.c
@@ -0,0 +1,98 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+#include <stdio.h>
+#include <stdlib.h>
+#include <string.h>
+
+#include "org_apache_hadoop.h"
+#include "isal_load.h"
+#include "erasure_code.h"
+#include "jni_common.h"
+
+void loadLib(JNIEnv *env) {
+  char errMsg[1024];
+  load_erasurecode_lib(errMsg, sizeof(errMsg));
+  if (strlen(errMsg) > 0) {
+    THROW(env, "java/lang/UnsatisfiedLinkError", errMsg);
+  }
+}
+
+void setCoder(JNIEnv* env, jobject thiz, IsalCoder* pCoder) {
+  jclass clazz = (*env)->GetObjectClass(env, thiz);
+  jfieldID __coderState = (*env)->GetFieldID(env, clazz, "__native_coder", "J");
+  (*env)->SetLongField(env, thiz, __coderState, (jlong) pCoder);
+}
+
+IsalCoder* getCoder(JNIEnv* env, jobject thiz) {
+  jclass clazz = (*env)->GetObjectClass(env, thiz);
+
+  jfieldID __verbose = (*env)->GetFieldID(env, clazz, "__native_verbose", "J");
+  int verbose = (int)(*env)->GetIntField(env, thiz, __verbose);
+
+  jfieldID __coderState = (*env)->GetFieldID(env, clazz, "__native_coder", "J");
+  IsalCoder* pCoder = (IsalCoder*)(*env)->GetLongField(env,
+                                                       thiz, __coderState);
+  pCoder->verbose = verbose;
+
+  return pCoder;
+}
+
+void getInputs(JNIEnv *env, jobjectArray inputs, jintArray inputOffsets,
+                              unsigned char** destInputs, int num) {
+  int numInputs = (*env)->GetArrayLength(env, inputs);
+  int* tmpInputOffsets;
+  int i;
+  jobject byteBuffer;
+
+  if (numInputs != num) {
+    THROW(env, "java/lang/InternalError", "Invalid inputs");
+  }
+
+  tmpInputOffsets = (int*)(*env)->GetIntArrayElements(env,
+                                                      inputOffsets, NULL);
+  for (i = 0; i < numInputs; i++) {
+    byteBuffer = (*env)->GetObjectArrayElement(env, inputs, i);
+    if (byteBuffer != NULL) {
+      destInputs[i] = (unsigned char *)((*env)->GetDirectBufferAddress(env,
+                                                                byteBuffer));
+      destInputs[i] += tmpInputOffsets[i];
+    } else {
+      destInputs[i] = NULL;
+    }
+  }
+}
+
+void getOutputs(JNIEnv *env, jobjectArray outputs, jintArray outputOffsets,
+                              unsigned char** destOutputs, int num) {
+  int numOutputs = (*env)->GetArrayLength(env, outputs);
+  int i, *tmpOutputOffsets;
+  jobject byteBuffer;
+
+  if (numOutputs != num) {
+    THROW(env, "java/lang/InternalError", "Invalid outputs");
+  }
+
+  tmpOutputOffsets = (int*)(*env)->GetIntArrayElements(env,
+                                                          outputOffsets, NULL);
+  for (i = 0; i < numOutputs; i++) {
+    byteBuffer = (*env)->GetObjectArrayElement(env, outputs, i);
+    destOutputs[i] = (unsigned char *)((*env)->GetDirectBufferAddress(env,
+                                                                  byteBuffer));
+    destOutputs[i] += tmpOutputOffsets[i];
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/658ee95f/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/erasurecode/jni_common.h
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/erasurecode/jni_common.h b/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/erasurecode/jni_common.h
new file mode 100644
index 0000000..a9f62a0
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/erasurecode/jni_common.h
@@ -0,0 +1,42 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#ifndef _JNI_CODER_COMMON_H_
+#define _JNI_CODER_COMMON_H_
+
+#include <stdio.h>
+#include <stdlib.h>
+#include <string.h>
+
+#include <jni.h>
+
+#include "erasure_coder.h"
+
+void loadLib(JNIEnv *env);
+
+void setCoder(JNIEnv* env, jobject thiz, IsalCoder* coder);
+
+IsalCoder* getCoder(JNIEnv* env, jobject thiz);
+
+void getInputs(JNIEnv *env, jobjectArray inputs, jintArray inputOffsets,
+                              unsigned char** destInputs, int num);
+
+void getOutputs(JNIEnv *env, jobjectArray outputs, jintArray outputOffsets,
+                              unsigned char** destOutputs, int num);
+
+#endif //_JNI_CODER_COMMON_H_
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/658ee95f/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/erasurecode/jni_erasure_code_native.c
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/erasurecode/jni_erasure_code_native.c b/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/erasurecode/jni_erasure_code_native.c
new file mode 100644
index 0000000..eb09e73
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/erasurecode/jni_erasure_code_native.c
@@ -0,0 +1,45 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#include <stdio.h>
+#include <stdlib.h>
+#include <string.h>
+
+#include "org_apache_hadoop.h"
+#include "jni_common.h"
+#include "org_apache_hadoop_io_erasurecode_ErasureCodeNative.h"
+
+#ifdef UNIX
+#include "config.h"
+#endif
+
+JNIEXPORT void JNICALL
+Java_org_apache_hadoop_io_erasurecode_ErasureCodeNative_loadLibrary
+(JNIEnv *env, jclass myclass) {
+  loadLib(env);
+}
+
+JNIEXPORT jstring JNICALL
+Java_org_apache_hadoop_io_erasurecode_ErasureCodeNative_getLibraryName
+(JNIEnv *env, jclass myclass) {
+  char* libName = get_library_name();
+  if (libName == NULL) {
+    libName = "Unavailable";
+  }
+  return (*env)->NewStringUTF(env, libName);
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/658ee95f/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/erasurecode/jni_rs_decoder.c
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/erasurecode/jni_rs_decoder.c b/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/erasurecode/jni_rs_decoder.c
new file mode 100644
index 0000000..eb4b903
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/erasurecode/jni_rs_decoder.c
@@ -0,0 +1,72 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#include <stdio.h>
+#include <stdlib.h>
+#include <string.h>
+
+#include "org_apache_hadoop.h"
+#include "erasure_code.h"
+#include "gf_util.h"
+#include "jni_common.h"
+#include "org_apache_hadoop_io_erasurecode_rawcoder_NativeRSRawDecoder.h"
+
+typedef struct _RSDecoder {
+  IsalDecoder decoder;
+  unsigned char* inputs[MMAX];
+  unsigned char* outputs[MMAX];
+} RSDecoder;
+
+JNIEXPORT void JNICALL
+Java_org_apache_hadoop_io_erasurecode_rawcoder_NativeRSRawDecoder_initImpl(
+JNIEnv *env, jobject thiz, jint numDataUnits, jint numParityUnits) {
+  RSDecoder* rsDecoder = (RSDecoder*)malloc(sizeof(RSDecoder));
+  memset(rsDecoder, 0, sizeof(*rsDecoder));
+  initDecoder(&rsDecoder->decoder, (int)numDataUnits, (int)numParityUnits);
+
+  setCoder(env, thiz, &rsDecoder->decoder.coder);
+}
+
+JNIEXPORT void JNICALL
+Java_org_apache_hadoop_io_erasurecode_rawcoder_NativeRSRawDecoder_decodeImpl(
+JNIEnv *env, jobject thiz, jobjectArray inputs, jintArray inputOffsets,
+jint dataLen, jintArray erasedIndexes, jobjectArray outputs,
+jintArray outputOffsets) {
+  RSDecoder* rsDecoder = (RSDecoder*)getCoder(env, thiz);
+
+  int numDataUnits = rsDecoder->decoder.coder.numDataUnits;
+  int numParityUnits = rsDecoder->decoder.coder.numParityUnits;
+  int chunkSize = (int)dataLen;
+
+  int* tmpErasedIndexes = (int*)(*env)->GetIntArrayElements(env,
+                                                        erasedIndexes, NULL);
+  int numErased = (*env)->GetArrayLength(env, erasedIndexes);
+  getInputs(env, inputs, inputOffsets, rsDecoder->inputs,
+                                               numDataUnits + numParityUnits);
+  getOutputs(env, outputs, outputOffsets, rsDecoder->outputs, numErased);
+
+  decode(&rsDecoder->decoder, rsDecoder->inputs, tmpErasedIndexes,
+                           numErased, rsDecoder->outputs, chunkSize);
+}
+
+JNIEXPORT void JNICALL
+Java_org_apache_hadoop_io_erasurecode_rawcoder_NativeRSRawDecoder_destroyImpl(
+JNIEnv *env, jobject thiz) {
+  RSDecoder* rsDecoder = (RSDecoder*)getCoder(env, thiz);
+  free(rsDecoder);
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/658ee95f/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/erasurecode/jni_rs_encoder.c
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/erasurecode/jni_rs_encoder.c b/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/erasurecode/jni_rs_encoder.c
new file mode 100644
index 0000000..6c477ed
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/erasurecode/jni_rs_encoder.c
@@ -0,0 +1,66 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#include <stdio.h>
+#include <stdlib.h>
+#include <string.h>
+
+#include "org_apache_hadoop.h"
+#include "erasure_code.h"
+#include "gf_util.h"
+#include "jni_common.h"
+#include "org_apache_hadoop_io_erasurecode_rawcoder_NativeRSRawEncoder.h"
+
+typedef struct _RSEncoder {
+  IsalEncoder encoder;
+  unsigned char* inputs[MMAX];
+  unsigned char* outputs[MMAX];
+} RSEncoder;
+
+JNIEXPORT void JNICALL
+Java_org_apache_hadoop_io_erasurecode_rawcoder_NativeRSRawEncoder_initImpl(
+JNIEnv *env, jobject thiz, jint numDataUnits, jint numParityUnits) {
+  RSEncoder* rsEncoder = (RSEncoder*)malloc(sizeof(RSEncoder));
+  memset(rsEncoder, 0, sizeof(*rsEncoder));
+  initEncoder(&rsEncoder->encoder, (int)numDataUnits, (int)numParityUnits);
+
+  setCoder(env, thiz, &rsEncoder->encoder.coder);
+}
+
+JNIEXPORT void JNICALL
+Java_org_apache_hadoop_io_erasurecode_rawcoder_NativeRSRawEncoder_encodeImpl(
+JNIEnv *env, jobject thiz, jobjectArray inputs, jintArray inputOffsets,
+jint dataLen, jobjectArray outputs, jintArray outputOffsets) {
+  RSEncoder* rsEncoder = (RSEncoder*)getCoder(env, thiz);
+
+  int numDataUnits = rsEncoder->encoder.coder.numDataUnits;
+  int numParityUnits = rsEncoder->encoder.coder.numParityUnits;
+  int chunkSize = (int)dataLen;
+
+  getInputs(env, inputs, inputOffsets, rsEncoder->inputs, numDataUnits);
+  getOutputs(env, outputs, outputOffsets, rsEncoder->outputs, numParityUnits);
+
+  encode(&rsEncoder->encoder, rsEncoder->inputs, rsEncoder->outputs, chunkSize);
+}
+
+JNIEXPORT void JNICALL
+Java_org_apache_hadoop_io_erasurecode_rawcoder_NativeRSRawEncoder_destroyImpl(
+JNIEnv *env, jobject thiz) {
+  RSEncoder* rsEncoder = (RSEncoder*)getCoder(env, thiz);
+  free(rsEncoder);
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/658ee95f/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/erasurecode/org_apache_hadoop_io_erasurecode_ErasureCodeNative.h
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/erasurecode/org_apache_hadoop_io_erasurecode_ErasureCodeNative.h b/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/erasurecode/org_apache_hadoop_io_erasurecode_ErasureCodeNative.h
new file mode 100644
index 0000000..d8ff3a0
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/erasurecode/org_apache_hadoop_io_erasurecode_ErasureCodeNative.h
@@ -0,0 +1,29 @@
+/* DO NOT EDIT THIS FILE - it is machine generated */
+#include <jni.h>
+/* Header for class org_apache_hadoop_io_erasurecode_ErasureCodeNative */
+
+#ifndef _Included_org_apache_hadoop_io_erasurecode_ErasureCodeNative
+#define _Included_org_apache_hadoop_io_erasurecode_ErasureCodeNative
+#ifdef __cplusplus
+extern "C" {
+#endif
+/*
+ * Class:     org_apache_hadoop_io_erasurecode_ErasureCodeNative
+ * Method:    loadLibrary
+ * Signature: ()V
+ */
+JNIEXPORT void JNICALL Java_org_apache_hadoop_io_erasurecode_ErasureCodeNative_loadLibrary
+  (JNIEnv *, jclass);
+
+/*
+ * Class:     org_apache_hadoop_io_erasurecode_ErasureCodeNative
+ * Method:    getLibraryName
+ * Signature: ()Ljava/lang/String;
+ */
+JNIEXPORT jstring JNICALL Java_org_apache_hadoop_io_erasurecode_ErasureCodeNative_getLibraryName
+  (JNIEnv *, jclass);
+
+#ifdef __cplusplus
+}
+#endif
+#endif


[28/46] hadoop git commit: HDFS-9857. Erasure Coding: Rename replication-based names in BlockManager to more generic [part-1]. Contributed by Rakesh R.

Posted by ae...@apache.org.
HDFS-9857. Erasure Coding: Rename replication-based names in BlockManager to more generic [part-1]. 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/32d043d9
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/32d043d9
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/32d043d9

Branch: refs/heads/HDFS-7240
Commit: 32d043d9c5f4615058ea4f65a58ba271ba47fcb5
Parents: 605fdcb
Author: Zhe Zhang <ze...@zezhang-ld1.linkedin.biz>
Authored: Wed Mar 16 16:53:58 2016 -0700
Committer: Zhe Zhang <ze...@zezhang-ld1.linkedin.biz>
Committed: Wed Mar 16 16:53:58 2016 -0700

----------------------------------------------------------------------
 .../server/blockmanagement/BlockManager.java    | 268 +++++------
 .../blockmanagement/DecommissionManager.java    |  30 +-
 .../blockmanagement/LowRedundancyBlocks.java    | 458 +++++++++++++++++++
 .../blockmanagement/UnderReplicatedBlocks.java  | 448 ------------------
 .../blockmanagement/BlockManagerTestUtil.java   |   2 +-
 .../blockmanagement/TestBlockManager.java       |  20 +-
 .../TestLowRedundancyBlockQueues.java           | 182 ++++++++
 .../blockmanagement/TestPendingReplication.java |  14 +-
 .../blockmanagement/TestReplicationPolicy.java  | 158 +++----
 .../TestUnderReplicatedBlockQueues.java         | 182 --------
 .../hdfs/server/namenode/TestMetaSave.java      |   2 +-
 11 files changed, 891 insertions(+), 873 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/32d043d9/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 6ed102c..66ab789 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
@@ -149,7 +149,7 @@ public class BlockManager implements BlockStatsMXBean {
 
   private volatile long pendingReplicationBlocksCount = 0L;
   private volatile long corruptReplicaBlocksCount = 0L;
-  private volatile long underReplicatedBlocksCount = 0L;
+  private volatile long lowRedundancyBlocksCount = 0L;
   private volatile long scheduledReplicationBlocksCount = 0L;
 
   /** flag indicating whether replication queues have been initialized */
@@ -166,7 +166,7 @@ public class BlockManager implements BlockStatsMXBean {
   }
   /** Used by metrics */
   public long getUnderReplicatedBlocksCount() {
-    return underReplicatedBlocksCount;
+    return lowRedundancyBlocksCount;
   }
   /** Used by metrics */
   public long getCorruptReplicaBlocksCount() {
@@ -250,9 +250,10 @@ public class BlockManager implements BlockStatsMXBean {
 
   /**
    * Store set of Blocks that need to be replicated 1 or more times.
-   * We also store pending replication-orders.
+   * We also store pending reconstruction-orders.
    */
-  public final UnderReplicatedBlocks neededReplications = new UnderReplicatedBlocks();
+  public final LowRedundancyBlocks neededReconstruction =
+      new LowRedundancyBlocks();
 
   @VisibleForTesting
   final PendingReplicationBlocks pendingReplications;
@@ -294,20 +295,20 @@ public class BlockManager implements BlockStatsMXBean {
   private boolean shouldPostponeBlocksFromFuture = false;
 
   /**
-   * Process replication queues asynchronously to allow namenode safemode exit
-   * and failover to be faster. HDFS-5496
+   * Process reconstruction queues asynchronously to allow namenode safemode
+   * exit and failover to be faster. HDFS-5496.
    */
-  private Daemon replicationQueuesInitializer = null;
+  private Daemon reconstructionQueuesInitializer = null;
   /**
-   * Number of blocks to process asychronously for replication queues
+   * Number of blocks to process asychronously for reconstruction queues
    * initialization once aquired the namesystem lock. Remaining blocks will be
    * processed again after aquiring lock again.
    */
   private int numBlocksPerIteration;
   /**
-   * Progress of the Replication queues initialisation.
+   * Progress of the Reconstruction queues initialisation.
    */
-  private double replicationQueuesInitProgress = 0.0;
+  private double reconstructionQueuesInitProgress = 0.0;
 
   /** for block replicas placement */
   private BlockPlacementPolicies placementPolicies;
@@ -576,12 +577,12 @@ public class BlockManager implements BlockStatsMXBean {
     out.println("Live Datanodes: " + live.size());
     out.println("Dead Datanodes: " + dead.size());
     //
-    // Dump contents of neededReplication
+    // Dump contents of neededReconstruction
     //
-    synchronized (neededReplications) {
-      out.println("Metasave: Blocks waiting for replication: " + 
-                  neededReplications.size());
-      for (Block block : neededReplications) {
+    synchronized (neededReconstruction) {
+      out.println("Metasave: Blocks waiting for reconstruction: "
+          + neededReconstruction.size());
+      for (Block block : neededReconstruction) {
         dumpBlockMeta(block, out);
       }
     }
@@ -616,7 +617,7 @@ public class BlockManager implements BlockStatsMXBean {
     // source node returned is not used
     chooseSourceDatanodes(getStoredBlock(block), containingNodes,
         containingLiveReplicasNodes, numReplicas,
-        new LinkedList<Byte>(), UnderReplicatedBlocks.LEVEL);
+        new LinkedList<Byte>(), LowRedundancyBlocks.LEVEL);
     
     // containingLiveReplicasNodes can include READ_ONLY_SHARED replicas which are 
     // not included in the numReplicas.liveReplicas() count
@@ -849,9 +850,9 @@ public class BlockManager implements BlockStatsMXBean {
     // is happening
     bc.convertLastBlockToUC(lastBlock, targets);
 
-    // Remove block from replication queue.
+    // Remove block from reconstruction queue.
     NumberReplicas replicas = countNodes(lastBlock);
-    neededReplications.remove(lastBlock, replicas.liveReplicas(),
+    neededReconstruction.remove(lastBlock, replicas.liveReplicas(),
         replicas.readOnlyReplicas(),
         replicas.decommissionedAndDecommissioning(), getReplication(lastBlock));
     pendingReplications.remove(lastBlock);
@@ -1365,8 +1366,8 @@ public class BlockManager implements BlockStatsMXBean {
       // the block is over-replicated so invalidate the replicas immediately
       invalidateBlock(b, node, numberOfReplicas);
     } else if (isPopulatingReplQueues()) {
-      // add the block to neededReplication
-      updateNeededReplications(b.getStored(), -1, 0);
+      // add the block to neededReconstruction
+      updateNeededReconstructions(b.getStored(), -1, 0);
     }
   }
 
@@ -1418,13 +1419,13 @@ public class BlockManager implements BlockStatsMXBean {
   
   void updateState() {
     pendingReplicationBlocksCount = pendingReplications.size();
-    underReplicatedBlocksCount = neededReplications.size();
+    lowRedundancyBlocksCount = neededReconstruction.size();
     corruptReplicaBlocksCount = corruptReplicas.size();
   }
 
-  /** Return number of under-replicated but not missing blocks */
+  /** Return number of low redundancy blocks but not missing blocks. */
   public int getUnderReplicatedNotMissingBlocks() {
-    return neededReplications.getUnderReplicatedBlockCount();
+    return neededReconstruction.getLowRedundancyBlockCount();
   }
   
   /**
@@ -1452,25 +1453,26 @@ public class BlockManager implements BlockStatsMXBean {
   }
 
   /**
-   * Scan blocks in {@link #neededReplications} and assign reconstruction
+   * Scan blocks in {@link #neededReconstruction} and assign reconstruction
    * (replication or erasure coding) work to data-nodes they belong to.
    *
    * The number of process blocks equals either twice the number of live
-   * data-nodes or the number of under-replicated blocks whichever is less.
+   * data-nodes or the number of low redundancy blocks whichever is less.
    *
-   * @return number of blocks scheduled for replication during this iteration.
+   * @return number of blocks scheduled for reconstruction during this
+   *         iteration.
    */
   int computeBlockReconstructionWork(int blocksToProcess) {
-    List<List<BlockInfo>> blocksToReplicate = null;
+    List<List<BlockInfo>> blocksToReconstruct = null;
     namesystem.writeLock();
     try {
-      // Choose the blocks to be replicated
-      blocksToReplicate = neededReplications
-          .chooseUnderReplicatedBlocks(blocksToProcess);
+      // Choose the blocks to be reconstructed
+      blocksToReconstruct = neededReconstruction
+          .chooseLowRedundancyBlocks(blocksToProcess);
     } finally {
       namesystem.writeUnlock();
     }
-    return computeReconstructionWorkForBlocks(blocksToReplicate);
+    return computeReconstructionWorkForBlocks(blocksToReconstruct);
   }
 
   /**
@@ -1489,7 +1491,7 @@ public class BlockManager implements BlockStatsMXBean {
     // Step 1: categorize at-risk blocks into replication and EC tasks
     namesystem.writeLock();
     try {
-      synchronized (neededReplications) {
+      synchronized (neededReconstruction) {
         for (int priority = 0; priority < blocksToReconstruct
             .size(); priority++) {
           for (BlockInfo block : blocksToReconstruct.get(priority)) {
@@ -1533,7 +1535,7 @@ public class BlockManager implements BlockStatsMXBean {
           continue;
         }
 
-        synchronized (neededReplications) {
+        synchronized (neededReconstruction) {
           if (validateReconstructionWork(rw)) {
             scheduledWork++;
           }
@@ -1544,7 +1546,7 @@ public class BlockManager implements BlockStatsMXBean {
     }
 
     if (blockLog.isDebugEnabled()) {
-      // log which blocks have been scheduled for replication
+      // log which blocks have been scheduled for reconstruction
       for(BlockReconstructionWork rw : reconWork){
         DatanodeStorageInfo[] targets = rw.getTargets();
         if (targets != null && targets.length != 0) {
@@ -1558,8 +1560,9 @@ public class BlockManager implements BlockStatsMXBean {
         }
       }
 
-      blockLog.debug("BLOCK* neededReplications = {} pendingReplications = {}",
-          neededReplications.size(), pendingReplications.size());
+      blockLog.debug(
+          "BLOCK* neededReconstruction = {} pendingReplications = {}",
+          neededReconstruction.size(), pendingReplications.size());
     }
 
     return scheduledWork;
@@ -1576,8 +1579,8 @@ public class BlockManager implements BlockStatsMXBean {
       int priority) {
     // skip abandoned block or block reopened for append
     if (block.isDeleted() || !block.isCompleteOrCommitted()) {
-      // remove from neededReplications
-      neededReplications.remove(block, priority);
+      // remove from neededReconstruction
+      neededReconstruction.remove(block, priority);
       return null;
     }
 
@@ -1605,8 +1608,8 @@ public class BlockManager implements BlockStatsMXBean {
     int pendingNum = pendingReplications.getNumReplicas(block);
     if (hasEnoughEffectiveReplicas(block, numReplicas, pendingNum,
         requiredReplication)) {
-      neededReplications.remove(block, priority);
-      blockLog.debug("BLOCK* Removing {} from neededReplications as" +
+      neededReconstruction.remove(block, priority);
+      blockLog.debug("BLOCK* Removing {} from neededReconstruction as" +
           " it has enough replicas", block);
       return null;
     }
@@ -1662,7 +1665,7 @@ public class BlockManager implements BlockStatsMXBean {
     // Recheck since global lock was released
     // skip abandoned block or block reopened for append
     if (block.isDeleted() || !block.isCompleteOrCommitted()) {
-      neededReplications.remove(block, priority);
+      neededReconstruction.remove(block, priority);
       rw.resetTargets();
       return false;
     }
@@ -1673,7 +1676,7 @@ public class BlockManager implements BlockStatsMXBean {
     final int pendingNum = pendingReplications.getNumReplicas(block);
     if (hasEnoughEffectiveReplicas(block, numReplicas, pendingNum,
         requiredReplication)) {
-      neededReplications.remove(block, priority);
+      neededReconstruction.remove(block, priority);
       rw.resetTargets();
       blockLog.debug("BLOCK* Removing {} from neededReplications as" +
           " it has enough replicas", block);
@@ -1705,9 +1708,9 @@ public class BlockManager implements BlockStatsMXBean {
         + "pendingReplications", block);
 
     int numEffectiveReplicas = numReplicas.liveReplicas() + pendingNum;
-    // remove from neededReplications
+    // remove from neededReconstruction
     if(numEffectiveReplicas + targets.length >= requiredReplication) {
-      neededReplications.remove(block, priority);
+      neededReconstruction.remove(block, priority);
     }
     return true;
   }
@@ -1852,7 +1855,7 @@ public class BlockManager implements BlockStatsMXBean {
         continue;
       }
 
-      if(priority != UnderReplicatedBlocks.QUEUE_HIGHEST_PRIORITY 
+      if (priority != LowRedundancyBlocks.QUEUE_HIGHEST_PRIORITY
           && !node.isDecommissionInProgress() 
           && node.getNumberOfBlocksToBeReplicated() >= maxReplicationStreams) {
         continue; // already reached replication limit
@@ -1905,9 +1908,10 @@ public class BlockManager implements BlockStatsMXBean {
             continue;
           }
           NumberReplicas num = countNodes(timedOutItems[i]);
-          if (isNeededReplication(bi, num.liveReplicas())) {
-            neededReplications.add(bi, num.liveReplicas(), num.readOnlyReplicas(),
-                num.decommissionedAndDecommissioning(), getReplication(bi));
+          if (isNeededReconstruction(bi, num.liveReplicas())) {
+            neededReconstruction.add(bi, num.liveReplicas(),
+                num.readOnlyReplicas(), num.decommissionedAndDecommissioning(),
+                getReplication(bi));
           }
         }
       } finally {
@@ -2777,7 +2781,7 @@ public class BlockManager implements BlockStatsMXBean {
    * intended for use with initial block report at startup. If not in startup
    * safe mode, will call standard addStoredBlock(). Assumes this method is
    * called "immediately" so there is no need to refresh the storedBlock from
-   * blocksMap. Doesn't handle underReplication/overReplication, or worry about
+   * blocksMap. Doesn't handle low redundancy/extra redundancy, or worry about
    * pendingReplications or corruptReplicas, because it's in startup safe mode.
    * Doesn't log every block, because there are typically millions of them.
    * 
@@ -2812,7 +2816,7 @@ public class BlockManager implements BlockStatsMXBean {
 
   /**
    * Modify (block-->datanode) map. Remove block from set of
-   * needed replications if this takes care of the problem.
+   * needed reconstruction if this takes care of the problem.
    * @return the block that is stored in blocksMap.
    */
   private Block addStoredBlock(final BlockInfo block,
@@ -2890,24 +2894,25 @@ public class BlockManager implements BlockStatsMXBean {
       return storedBlock;
     }
 
-    // do not try to handle over/under-replicated blocks during first safe mode
+    // do not try to handle extra/low redundancy blocks during first safe mode
     if (!isPopulatingReplQueues()) {
       return storedBlock;
     }
 
-    // handle underReplication/overReplication
+    // handle low redundancy/extra redundancy
     short fileReplication = getExpectedReplicaNum(storedBlock);
-    if (!isNeededReplication(storedBlock, numCurrentReplica)) {
-      neededReplications.remove(storedBlock, numCurrentReplica,
+    if (!isNeededReconstruction(storedBlock, numCurrentReplica)) {
+      neededReconstruction.remove(storedBlock, numCurrentReplica,
           num.readOnlyReplicas(),
           num.decommissionedAndDecommissioning(), fileReplication);
     } else {
-      updateNeededReplications(storedBlock, curReplicaDelta, 0);
+      updateNeededReconstructions(storedBlock, curReplicaDelta, 0);
     }
-    if (shouldProcessOverReplicated(num, fileReplication)) {
-      processOverReplicatedBlock(storedBlock, fileReplication, node, delNodeHint);
+    if (shouldProcessExtraRedundancy(num, fileReplication)) {
+      processExtraRedundancyBlock(storedBlock, fileReplication, node,
+          delNodeHint);
     }
-    // If the file replication has reached desired value
+    // If the file redundancy has reached desired value
     // we can remove any corrupt replicas the block may have
     int corruptReplicasCount = corruptReplicas.numCorruptReplicas(storedBlock);
     int numCorruptNodes = num.corruptReplicas();
@@ -2922,7 +2927,7 @@ public class BlockManager implements BlockStatsMXBean {
     return storedBlock;
   }
 
-  private boolean shouldProcessOverReplicated(NumberReplicas num,
+  private boolean shouldProcessExtraRedundancy(NumberReplicas num,
       int expectedNum) {
     final int numCurrent = num.liveReplicas();
     return numCurrent > expectedNum ||
@@ -2972,42 +2977,44 @@ public class BlockManager implements BlockStatsMXBean {
 
   /**
    * For each block in the name-node verify whether it belongs to any file,
-   * over or under replicated. Place it into the respective queue.
+   * extra or low redundancy. Place it into the respective queue.
    */
   public void processMisReplicatedBlocks() {
     assert namesystem.hasWriteLock();
-    stopReplicationInitializer();
-    neededReplications.clear();
-    replicationQueuesInitializer = new Daemon() {
+    stopReconstructionInitializer();
+    neededReconstruction.clear();
+    reconstructionQueuesInitializer = new Daemon() {
 
       @Override
       public void run() {
         try {
           processMisReplicatesAsync();
         } catch (InterruptedException ie) {
-          LOG.info("Interrupted while processing replication queues.");
+          LOG.info("Interrupted while processing reconstruction queues.");
         } catch (Exception e) {
-          LOG.error("Error while processing replication queues async", e);
+          LOG.error("Error while processing reconstruction queues async", e);
         }
       }
     };
-    replicationQueuesInitializer.setName("Replication Queue Initializer");
-    replicationQueuesInitializer.start();
+    reconstructionQueuesInitializer
+        .setName("Reconstruction Queue Initializer");
+    reconstructionQueuesInitializer.start();
   }
 
   /*
-   * Stop the ongoing initialisation of replication queues
+   * Stop the ongoing initialisation of reconstruction queues
    */
-  private void stopReplicationInitializer() {
-    if (replicationQueuesInitializer != null) {
-      replicationQueuesInitializer.interrupt();
+  private void stopReconstructionInitializer() {
+    if (reconstructionQueuesInitializer != null) {
+      reconstructionQueuesInitializer.interrupt();
       try {
-        replicationQueuesInitializer.join();
+        reconstructionQueuesInitializer.join();
       } catch (final InterruptedException e) {
-        LOG.warn("Interrupted while waiting for replicationQueueInitializer. Returning..");
+        LOG.warn("Interrupted while waiting for "
+            + "reconstructionQueueInitializer. Returning..");
         return;
       } finally {
-        replicationQueuesInitializer = null;
+        reconstructionQueuesInitializer = null;
       }
     }
   }
@@ -3025,7 +3032,7 @@ public class BlockManager implements BlockStatsMXBean {
     long startTimeMisReplicatedScan = Time.monotonicNow();
     Iterator<BlockInfo> blocksItr = blocksMap.getBlocks().iterator();
     long totalBlocks = blocksMap.size();
-    replicationQueuesInitProgress = 0;
+    reconstructionQueuesInitProgress = 0;
     long totalProcessed = 0;
     long sleepDuration =
         Math.max(1, Math.min(numBlocksPerIteration/1000, 10000));
@@ -3067,7 +3074,7 @@ public class BlockManager implements BlockStatsMXBean {
         totalProcessed += processed;
         // there is a possibility that if any of the blocks deleted/added during
         // initialisation, then progress might be different.
-        replicationQueuesInitProgress = Math.min((double) totalProcessed
+        reconstructionQueuesInitProgress = Math.min((double) totalProcessed
             / totalBlocks, 1.0);
 
         if (!blocksItr.hasNext()) {
@@ -3097,12 +3104,12 @@ public class BlockManager implements BlockStatsMXBean {
   }
 
   /**
-   * Get the progress of the Replication queues initialisation
+   * Get the progress of the reconstruction queues initialisation
    * 
    * @return Returns values between 0 and 1 for the progress.
    */
-  public double getReplicationQueuesInitProgress() {
-    return replicationQueuesInitProgress;
+  public double getReconstructionQueuesInitProgress() {
+    return reconstructionQueuesInitProgress;
   }
 
   /**
@@ -3134,15 +3141,16 @@ public class BlockManager implements BlockStatsMXBean {
     short expectedReplication = getExpectedReplicaNum(block);
     NumberReplicas num = countNodes(block);
     final int numCurrentReplica = num.liveReplicas();
-    // add to under-replicated queue if need to be
-    if (isNeededReplication(block, numCurrentReplica)) {
-      if (neededReplications.add(block, numCurrentReplica, num.readOnlyReplicas(),
-          num.decommissionedAndDecommissioning(), expectedReplication)) {
+    // add to low redundancy queue if need to be
+    if (isNeededReconstruction(block, numCurrentReplica)) {
+      if (neededReconstruction.add(block, numCurrentReplica,
+          num.readOnlyReplicas(), num.decommissionedAndDecommissioning(),
+          expectedReplication)) {
         return MisReplicationResult.UNDER_REPLICATED;
       }
     }
 
-    if (shouldProcessOverReplicated(num, expectedReplication)) {
+    if (shouldProcessExtraRedundancy(num, expectedReplication)) {
       if (num.replicasOnStaleNodes() > 0) {
         // If any of the replicas of this block are on nodes that are
         // considered "stale", then these replicas may in fact have
@@ -3152,8 +3160,8 @@ public class BlockManager implements BlockStatsMXBean {
         return MisReplicationResult.POSTPONE;
       }
       
-      // over-replicated block
-      processOverReplicatedBlock(block, expectedReplication, null, null);
+      // extra redundancy block
+      processExtraRedundancyBlock(block, expectedReplication, null, null);
       return MisReplicationResult.OVER_REPLICATED;
     }
     
@@ -3167,12 +3175,12 @@ public class BlockManager implements BlockStatsMXBean {
       return;
     }
 
-    // update needReplication priority queues
+    // update neededReconstruction priority queues
     b.setReplication(newRepl);
-    updateNeededReplications(b, 0, newRepl - oldRepl);
+    updateNeededReconstructions(b, 0, newRepl - oldRepl);
 
     if (oldRepl > newRepl) {
-      processOverReplicatedBlock(b, newRepl, null, null);
+      processExtraRedundancyBlock(b, newRepl, null, null);
     }
   }
 
@@ -3181,7 +3189,7 @@ public class BlockManager implements BlockStatsMXBean {
    * If there are any extras, call chooseExcessReplicates() to
    * mark them in the excessReplicateMap.
    */
-  private void processOverReplicatedBlock(final BlockInfo block,
+  private void processExtraRedundancyBlock(final BlockInfo block,
       final short replication, final DatanodeDescriptor addedNode,
       DatanodeDescriptor delNodeHint) {
     assert namesystem.hasWriteLock();
@@ -3405,7 +3413,7 @@ public class BlockManager implements BlockStatsMXBean {
       //
       if (!storedBlock.isDeleted()) {
         bmSafeMode.decrementSafeBlockCount(storedBlock);
-        updateNeededReplications(storedBlock, -1, 0);
+        updateNeededReconstructions(storedBlock, -1, 0);
       }
 
       excessReplicas.remove(node, storedBlock);
@@ -3748,29 +3756,29 @@ public class BlockManager implements BlockStatsMXBean {
   
   /**
    * On stopping decommission, check if the node has excess replicas.
-   * If there are any excess replicas, call processOverReplicatedBlock().
-   * Process over replicated blocks only when active NN is out of safe mode.
+   * If there are any excess replicas, call processExtraRedundancyBlock().
+   * Process extra redundancy blocks only when active NN is out of safe mode.
    */
-  void processOverReplicatedBlocksOnReCommission(
+  void processExtraRedundancyBlocksOnReCommission(
       final DatanodeDescriptor srcNode) {
     if (!isPopulatingReplQueues()) {
       return;
     }
     final Iterator<BlockInfo> it = srcNode.getBlockIterator();
-    int numOverReplicated = 0;
+    int numExtraRedundancy = 0;
     while(it.hasNext()) {
       final BlockInfo block = it.next();
       int expectedReplication = this.getReplication(block);
       NumberReplicas num = countNodes(block);
-      if (shouldProcessOverReplicated(num, expectedReplication)) {
-        // over-replicated block
-        processOverReplicatedBlock(block, (short) expectedReplication, null,
+      if (shouldProcessExtraRedundancy(num, expectedReplication)) {
+        // extra redundancy block
+        processExtraRedundancyBlock(block, (short) expectedReplication, null,
             null);
-        numOverReplicated++;
+        numExtraRedundancy++;
       }
     }
-    LOG.info("Invalidated " + numOverReplicated + " over-replicated blocks on " +
-        srcNode + " during recommissioning");
+    LOG.info("Invalidated " + numExtraRedundancy
+        + " extra redundancy blocks on " + srcNode + " during recommissioning");
   }
 
   /**
@@ -3789,9 +3797,9 @@ public class BlockManager implements BlockStatsMXBean {
 
     updateState();
     if (pendingReplicationBlocksCount == 0 &&
-        underReplicatedBlocksCount == 0) {
-      LOG.info("Node {} is dead and there are no under-replicated" +
-          " blocks or blocks pending replication. Safe to decommission.",
+        lowRedundancyBlocksCount == 0) {
+      LOG.info("Node {} is dead and there are no low redundancy" +
+          " blocks or blocks pending reconstruction. Safe to decommission.",
           node);
       return true;
     }
@@ -3835,9 +3843,9 @@ public class BlockManager implements BlockStatsMXBean {
     block.setNumBytes(BlockCommand.NO_ACK);
     addToInvalidates(block);
     removeBlockFromMap(block);
-    // Remove the block from pendingReplications and neededReplications
+    // Remove the block from pendingReplications and neededReconstruction
     pendingReplications.remove(block);
-    neededReplications.remove(block, UnderReplicatedBlocks.LEVEL);
+    neededReconstruction.remove(block, LowRedundancyBlocks.LEVEL);
     if (postponedMisreplicatedBlocks.remove(block)) {
       postponedMisreplicatedBlocksCount.decrementAndGet();
     }
@@ -3859,8 +3867,8 @@ public class BlockManager implements BlockStatsMXBean {
         new Block(BlockIdManager.convertToStripedID(block.getBlockId())));
   }
 
-  /** updates a block in under replication queue */
-  private void updateNeededReplications(final BlockInfo block,
+  /** updates a block in needed reconstruction queue. */
+  private void updateNeededReconstructions(final BlockInfo block,
       final int curReplicasDelta, int expectedReplicasDelta) {
     namesystem.writeLock();
     try {
@@ -3869,14 +3877,14 @@ public class BlockManager implements BlockStatsMXBean {
       }
       NumberReplicas repl = countNodes(block);
       int curExpectedReplicas = getReplication(block);
-      if (isNeededReplication(block, repl.liveReplicas())) {
-        neededReplications.update(block, repl.liveReplicas(), repl.readOnlyReplicas(),
-            repl.decommissionedAndDecommissioning(), curExpectedReplicas,
-            curReplicasDelta, expectedReplicasDelta);
+      if (isNeededReconstruction(block, repl.liveReplicas())) {
+        neededReconstruction.update(block, repl.liveReplicas(),
+            repl.readOnlyReplicas(), repl.decommissionedAndDecommissioning(),
+            curExpectedReplicas, curReplicasDelta, expectedReplicasDelta);
       } else {
         int oldReplicas = repl.liveReplicas()-curReplicasDelta;
         int oldExpectedReplicas = curExpectedReplicas-expectedReplicasDelta;
-        neededReplications.remove(block, oldReplicas, repl.readOnlyReplicas(),
+        neededReconstruction.remove(block, oldReplicas, repl.readOnlyReplicas(),
             repl.decommissionedAndDecommissioning(), oldExpectedReplicas);
       }
     } finally {
@@ -3885,10 +3893,10 @@ public class BlockManager implements BlockStatsMXBean {
   }
 
   /**
-   * Check replication of the blocks in the collection.
-   * If any block is needed replication, insert it into the replication queue.
+   * Check sufficient redundancy of the blocks in the collection. If any block
+   * is needed reconstruction, insert it into the reconstruction queue.
    * Otherwise, if the block is more than the expected replication factor,
-   * process it as an over replicated block.
+   * process it as an extra redundancy block.
    */
   public void checkReplication(BlockCollection bc) {
     for (BlockInfo block : bc.getBlocks()) {
@@ -3896,11 +3904,11 @@ public class BlockManager implements BlockStatsMXBean {
       final NumberReplicas n = countNodes(block);
       final int pending = pendingReplications.getNumReplicas(block);
       if (!hasEnoughEffectiveReplicas(block, n, pending, expected)) {
-        neededReplications.add(block, n.liveReplicas() + pending,
+        neededReconstruction.add(block, n.liveReplicas() + pending,
             n.readOnlyReplicas(),
             n.decommissionedAndDecommissioning(), expected);
-      } else if (shouldProcessOverReplicated(n, expected)) {
-        processOverReplicatedBlock(block, expected, null, null);
+      } else if (shouldProcessExtraRedundancy(n, expected)) {
+        processExtraRedundancyBlock(block, expected, null, null);
       }
     }
   }
@@ -3926,7 +3934,7 @@ public class BlockManager implements BlockStatsMXBean {
     try {
       // blocks should not be replicated or removed if safe mode is on
       if (namesystem.isInSafeMode()) {
-        LOG.debug("In safemode, not computing replication work");
+        LOG.debug("In safemode, not computing reconstruction work");
         return 0;
       }
       try {
@@ -3980,10 +3988,10 @@ public class BlockManager implements BlockStatsMXBean {
   }
 
   /**
-   * A block needs replication if the number of replicas is less than expected
-   * or if it does not have enough racks.
+   * A block needs reconstruction if the number of replicas is less than
+   * expected or if it does not have enough racks.
    */
-  boolean isNeededReplication(BlockInfo storedBlock, int current) {
+  boolean isNeededReconstruction(BlockInfo storedBlock, int current) {
     int expected = getExpectedReplicaNum(storedBlock);
     return storedBlock.isComplete()
         && (current < expected || !isPlacementPolicySatisfied(storedBlock));
@@ -3997,12 +4005,12 @@ public class BlockManager implements BlockStatsMXBean {
 
   public long getMissingBlocksCount() {
     // not locking
-    return this.neededReplications.getCorruptBlockSize();
+    return this.neededReconstruction.getCorruptBlockSize();
   }
 
   public long getMissingReplOneBlocksCount() {
     // not locking
-    return this.neededReplications.getCorruptReplOneBlockSize();
+    return this.neededReconstruction.getCorruptReplOneBlockSize();
   }
 
   public BlockInfo addBlockCollection(BlockInfo block,
@@ -4050,8 +4058,8 @@ public class BlockManager implements BlockStatsMXBean {
    * Return an iterator over the set of blocks for which there are no replicas.
    */
   public Iterator<BlockInfo> getCorruptReplicaBlockIterator() {
-    return neededReplications.iterator(
-        UnderReplicatedBlocks.QUEUE_WITH_CORRUPT_BLOCKS);
+    return neededReconstruction.iterator(
+        LowRedundancyBlocks.QUEUE_WITH_CORRUPT_BLOCKS);
   }
 
   /**
@@ -4070,7 +4078,7 @@ public class BlockManager implements BlockStatsMXBean {
 
   /** @return the size of UnderReplicatedBlocks */
   public int numOfUnderReplicatedBlocks() {
-    return neededReplications.size();
+    return neededReconstruction.size();
   }
 
   /**
@@ -4232,7 +4240,7 @@ public class BlockManager implements BlockStatsMXBean {
    * this NameNode.
    */
   public void clearQueues() {
-    neededReplications.clear();
+    neededReconstruction.clear();
     pendingReplications.clear();
     excessReplicas.clear();
     invalidateBlocks.clear();
@@ -4298,7 +4306,7 @@ public class BlockManager implements BlockStatsMXBean {
   }
 
   public void shutdown() {
-    stopReplicationInitializer();
+    stopReconstructionInitializer();
     blocksMap.close();
     MBeans.unregister(mxBeanName);
     mxBeanName = null;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/32d043d9/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 480670a..3b5f103 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
@@ -215,10 +215,10 @@ public class DecommissionManager {
     if (node.isDecommissionInProgress() || node.isDecommissioned()) {
       // Update DN stats maintained by HeartbeatManager
       hbManager.stopDecommission(node);
-      // Over-replicated blocks will be detected and processed when
+      // extra redundancy blocks will be detected and processed when
       // the dead node comes back and send in its full block report.
       if (node.isAlive()) {
-        blockManager.processOverReplicatedBlocksOnReCommission(node);
+        blockManager.processExtraRedundancyBlocksOnReCommission(node);
       }
       // Remove from tracking in DecommissionManager
       pendingNodes.remove(node);
@@ -513,9 +513,9 @@ public class DecommissionManager {
         final List<BlockInfo> insufficientList,
         boolean pruneReliableBlocks) {
       boolean firstReplicationLog = true;
-      int underReplicatedBlocks = 0;
+      int lowRedundancyBlocks = 0;
       int decommissionOnlyReplicas = 0;
-      int underReplicatedInOpenFiles = 0;
+      int lowRedundancyInOpenFiles = 0;
       while (it.hasNext()) {
         numBlocksChecked++;
         final BlockInfo block = it.next();
@@ -537,22 +537,22 @@ public class DecommissionManager {
         final NumberReplicas num = blockManager.countNodes(block);
         final int liveReplicas = num.liveReplicas();
 
-        // Schedule under-replicated blocks for replication if not already
+        // Schedule low redundancy blocks for reconstruction if not already
         // pending
-        if (blockManager.isNeededReplication(block, liveReplicas)) {
-          if (!blockManager.neededReplications.contains(block) &&
+        if (blockManager.isNeededReconstruction(block, liveReplicas)) {
+          if (!blockManager.neededReconstruction.contains(block) &&
               blockManager.pendingReplications.getNumReplicas(block) == 0 &&
               blockManager.isPopulatingReplQueues()) {
             // Process these blocks only when active NN is out of safe mode.
-            blockManager.neededReplications.add(block,
+            blockManager.neededReconstruction.add(block,
                 liveReplicas, num.readOnlyReplicas(),
                 num.decommissionedAndDecommissioning(),
                 blockManager.getExpectedReplicaNum(block));
           }
         }
 
-        // Even if the block is under-replicated, 
-        // it doesn't block decommission if it's sufficiently replicated
+        // Even if the block is without sufficient redundancy,
+        // it doesn't block decommission if has sufficient redundancy
         if (isSufficient(block, bc, num)) {
           if (pruneReliableBlocks) {
             it.remove();
@@ -560,7 +560,7 @@ public class DecommissionManager {
           continue;
         }
 
-        // We've found an insufficiently replicated block.
+        // We've found a block without sufficient redundancy.
         if (insufficientList != null) {
           insufficientList.add(block);
         }
@@ -571,18 +571,18 @@ public class DecommissionManager {
           firstReplicationLog = false;
         }
         // Update various counts
-        underReplicatedBlocks++;
+        lowRedundancyBlocks++;
         if (bc.isUnderConstruction()) {
-          underReplicatedInOpenFiles++;
+          lowRedundancyInOpenFiles++;
         }
         if ((liveReplicas == 0) && (num.decommissionedAndDecommissioning() > 0)) {
           decommissionOnlyReplicas++;
         }
       }
 
-      datanode.decommissioningStatus.set(underReplicatedBlocks,
+      datanode.decommissioningStatus.set(lowRedundancyBlocks,
           decommissionOnlyReplicas,
-          underReplicatedInOpenFiles);
+          lowRedundancyInOpenFiles);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/32d043d9/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/LowRedundancyBlocks.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/LowRedundancyBlocks.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/LowRedundancyBlocks.java
new file mode 100644
index 0000000..de8cf4e
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/LowRedundancyBlocks.java
@@ -0,0 +1,458 @@
+/**
+ * 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 java.util.ArrayList;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+
+import org.apache.hadoop.hdfs.server.namenode.NameNode;
+import org.apache.hadoop.hdfs.util.LightWeightLinkedSet;
+
+/**
+ * Keep prioritized queues of low redundant blocks.
+ * Blocks have redundancy priority, with priority
+ * {@link #QUEUE_HIGHEST_PRIORITY} indicating the highest priority.
+ * </p>
+ * Having a prioritised queue allows the {@link BlockManager} to select
+ * which blocks to replicate first -it tries to give priority to data
+ * that is most at risk or considered most valuable.
+ *
+ * <p/>
+ * The policy for choosing which priority to give added blocks
+ * is implemented in {@link #getPriority(BlockInfo, int, int, int, int)}.
+ * </p>
+ * <p>The queue order is as follows:</p>
+ * <ol>
+ *   <li>{@link #QUEUE_HIGHEST_PRIORITY}: the blocks that should be redundant
+ *   first. That is blocks with only one copy, or blocks with zero live
+ *   copies but a copy in a node being decommissioned. These blocks
+ *   are at risk of loss if the disk or server on which they
+ *   remain fails.</li>
+ *   <li>{@link #QUEUE_VERY_LOW_REDUNDANCY}: blocks that are very
+ *   under-replicated compared to their expected values. Currently
+ *   that means the ratio of the ratio of actual:expected means that
+ *   there is <i>less than</i> 1:3.</li>. These blocks may not be at risk,
+ *   but they are clearly considered "important".
+ *   <li>{@link #QUEUE_LOW_REDUNDANCY}: blocks that are also under
+ *   replicated, and the ratio of actual:expected is good enough that
+ *   they do not need to go into the {@link #QUEUE_VERY_LOW_REDUNDANCY}
+ *   queue.</li>
+ *   <li>{@link #QUEUE_REPLICAS_BADLY_DISTRIBUTED}: there are as least as
+ *   many copies of a block as required, but the blocks are not adequately
+ *   distributed. Loss of a rack/switch could take all copies off-line.</li>
+ *   <li>{@link #QUEUE_WITH_CORRUPT_BLOCKS} This is for blocks that are corrupt
+ *   and for which there are no-non-corrupt copies (currently) available.
+ *   The policy here is to keep those corrupt blocks replicated, but give
+ *   blocks that are not corrupt higher priority.</li>
+ * </ol>
+ */
+class LowRedundancyBlocks implements Iterable<BlockInfo> {
+  /** The total number of queues : {@value} */
+  static final int LEVEL = 5;
+  /** The queue with the highest priority: {@value} */
+  static final int QUEUE_HIGHEST_PRIORITY = 0;
+  /** The queue for blocks that are way below their expected value : {@value} */
+  static final int QUEUE_VERY_LOW_REDUNDANCY = 1;
+  /**
+   * The queue for "normally" without sufficient redundancy blocks : {@value}.
+   */
+  static final int QUEUE_LOW_REDUNDANCY = 2;
+  /** The queue for blocks that have the right number of replicas,
+   * but which the block manager felt were badly distributed: {@value}
+   */
+  static final int QUEUE_REPLICAS_BADLY_DISTRIBUTED = 3;
+  /** The queue for corrupt blocks: {@value} */
+  static final int QUEUE_WITH_CORRUPT_BLOCKS = 4;
+  /** the queues themselves */
+  private final List<LightWeightLinkedSet<BlockInfo>> priorityQueues
+      = new ArrayList<>(LEVEL);
+
+  /** The number of corrupt blocks with replication factor 1 */
+  private int corruptReplOneBlocks = 0;
+
+  /** Create an object. */
+  LowRedundancyBlocks() {
+    for (int i = 0; i < LEVEL; i++) {
+      priorityQueues.add(new LightWeightLinkedSet<BlockInfo>());
+    }
+  }
+
+  /**
+   * Empty the queues.
+   */
+  synchronized void clear() {
+    for (int i = 0; i < LEVEL; i++) {
+      priorityQueues.get(i).clear();
+    }
+    corruptReplOneBlocks = 0;
+  }
+
+  /** Return the total number of insufficient redundancy blocks. */
+  synchronized int size() {
+    int size = 0;
+    for (int i = 0; i < LEVEL; i++) {
+      size += priorityQueues.get(i).size();
+    }
+    return size;
+  }
+
+  /**
+   * Return the number of insufficiently redundant blocks excluding corrupt
+   * blocks.
+   */
+  synchronized int getLowRedundancyBlockCount() {
+    int size = 0;
+    for (int i = 0; i < LEVEL; i++) {
+      if (i != QUEUE_WITH_CORRUPT_BLOCKS) {
+        size += priorityQueues.get(i).size();
+      }
+    }
+    return size;
+  }
+
+  /** Return the number of corrupt blocks */
+  synchronized int getCorruptBlockSize() {
+    return priorityQueues.get(QUEUE_WITH_CORRUPT_BLOCKS).size();
+  }
+
+  /** Return the number of corrupt blocks with replication factor 1 */
+  synchronized int getCorruptReplOneBlockSize() {
+    return corruptReplOneBlocks;
+  }
+
+  /** Check if a block is in the neededReconstruction queue. */
+  synchronized boolean contains(BlockInfo block) {
+    for(LightWeightLinkedSet<BlockInfo> set : priorityQueues) {
+      if (set.contains(block)) {
+        return true;
+      }
+    }
+    return false;
+  }
+
+  /** Return the priority of a block
+   * @param curReplicas current number of replicas of the block
+   * @param expectedReplicas expected number of replicas of the block
+   * @return the priority for the blocks, between 0 and ({@link #LEVEL}-1)
+   */
+  private int getPriority(BlockInfo block,
+                          int curReplicas,
+                          int readOnlyReplicas,
+                          int decommissionedReplicas,
+                          int expectedReplicas) {
+    assert curReplicas >= 0 : "Negative replicas!";
+    if (curReplicas >= expectedReplicas) {
+      // Block has enough copies, but not enough racks
+      return QUEUE_REPLICAS_BADLY_DISTRIBUTED;
+    }
+    if (block.isStriped()) {
+      BlockInfoStriped sblk = (BlockInfoStriped) block;
+      return getPriorityStriped(curReplicas, decommissionedReplicas,
+          sblk.getRealDataBlockNum(), sblk.getParityBlockNum());
+    } else {
+      return getPriorityContiguous(curReplicas, readOnlyReplicas,
+          decommissionedReplicas, expectedReplicas);
+    }
+  }
+
+  private int getPriorityContiguous(int curReplicas, int readOnlyReplicas,
+      int decommissionedReplicas, int expectedReplicas) {
+    if (curReplicas == 0) {
+      // If there are zero non-decommissioned replicas but there are
+      // some decommissioned replicas, then assign them highest priority
+      if (decommissionedReplicas > 0) {
+        return QUEUE_HIGHEST_PRIORITY;
+      }
+      if (readOnlyReplicas > 0) {
+        // only has read-only replicas, highest risk
+        // since the read-only replicas may go down all together.
+        return QUEUE_HIGHEST_PRIORITY;
+      }
+      //all we have are corrupt blocks
+      return QUEUE_WITH_CORRUPT_BLOCKS;
+    } else if (curReplicas == 1) {
+      // only one replica, highest risk of loss
+      // highest priority
+      return QUEUE_HIGHEST_PRIORITY;
+    } else if ((curReplicas * 3) < expectedReplicas) {
+      //can only afford one replica loss
+      //this is considered very insufficiently redundant blocks.
+      return QUEUE_VERY_LOW_REDUNDANCY;
+    } else {
+      //add to the normal queue for insufficiently redundant blocks
+      return QUEUE_LOW_REDUNDANCY;
+    }
+  }
+
+  private int getPriorityStriped(int curReplicas, int decommissionedReplicas,
+      short dataBlkNum, short parityBlkNum) {
+    if (curReplicas < dataBlkNum) {
+      // There are some replicas on decommissioned nodes so it's not corrupted
+      if (curReplicas + decommissionedReplicas >= dataBlkNum) {
+        return QUEUE_HIGHEST_PRIORITY;
+      }
+      return QUEUE_WITH_CORRUPT_BLOCKS;
+    } else if (curReplicas == dataBlkNum) {
+      // highest risk of loss, highest priority
+      return QUEUE_HIGHEST_PRIORITY;
+    } else if ((curReplicas - dataBlkNum) * 3 < parityBlkNum + 1) {
+      // can only afford one replica loss
+      // this is considered very insufficiently redundant blocks.
+      return QUEUE_VERY_LOW_REDUNDANCY;
+    } else {
+      // add to the normal queue for insufficiently redundant blocks.
+      return QUEUE_LOW_REDUNDANCY;
+    }
+  }
+
+  /**
+   * Add a block to insufficiently redundant queue according to its priority.
+   *
+   * @param block a low redundancy block
+   * @param curReplicas current number of replicas of the block
+   * @param decomissionedReplicas the number of decommissioned replicas
+   * @param expectedReplicas expected number of replicas of the block
+   * @return true if the block was added to a queue.
+   */
+  synchronized boolean add(BlockInfo block,
+                           int curReplicas,
+                           int readOnlyReplicas,
+                           int decomissionedReplicas,
+                           int expectedReplicas) {
+    assert curReplicas >= 0 : "Negative replicas!";
+    final int priLevel = getPriority(block, curReplicas, readOnlyReplicas,
+        decomissionedReplicas, expectedReplicas);
+    if(priorityQueues.get(priLevel).add(block)) {
+      if (priLevel == QUEUE_WITH_CORRUPT_BLOCKS &&
+          expectedReplicas == 1) {
+        corruptReplOneBlocks++;
+      }
+      NameNode.blockStateChangeLog.debug(
+          "BLOCK* NameSystem.LowRedundancyBlock.add: {}"
+              + " has only {} replicas and need {} replicas so is added to"
+              + " neededReconstructions at priority level {}",
+          block, curReplicas, expectedReplicas, priLevel);
+
+      return true;
+    }
+    return false;
+  }
+
+  /** Remove a block from a low redundancy queue. */
+  synchronized boolean remove(BlockInfo block,
+                              int oldReplicas,
+                              int oldReadOnlyReplicas,
+                              int decommissionedReplicas,
+                              int oldExpectedReplicas) {
+    final int priLevel = getPriority(block, oldReplicas, oldReadOnlyReplicas,
+        decommissionedReplicas, oldExpectedReplicas);
+    boolean removedBlock = remove(block, priLevel);
+    if (priLevel == QUEUE_WITH_CORRUPT_BLOCKS &&
+        oldExpectedReplicas == 1 &&
+        removedBlock) {
+      corruptReplOneBlocks--;
+      assert corruptReplOneBlocks >= 0 :
+          "Number of corrupt blocks with replication factor 1 " +
+              "should be non-negative";
+    }
+    return removedBlock;
+  }
+
+  /**
+   * Remove a block from the low redundancy queues.
+   *
+   * The priLevel parameter is a hint of which queue to query
+   * first: if negative or &gt;= {@link #LEVEL} this shortcutting
+   * is not attmpted.
+   *
+   * If the block is not found in the nominated queue, an attempt is made to
+   * remove it from all queues.
+   *
+   * <i>Warning:</i> This is not a synchronized method.
+   * @param block block to remove
+   * @param priLevel expected privilege level
+   * @return true if the block was found and removed from one of the priority
+   *         queues
+   */
+  boolean remove(BlockInfo block, int priLevel) {
+    if(priLevel >= 0 && priLevel < LEVEL
+        && priorityQueues.get(priLevel).remove(block)) {
+      NameNode.blockStateChangeLog.debug(
+          "BLOCK* NameSystem.LowRedundancyBlock.remove: Removing block {}"
+              + " from priority queue {}",
+          block, priLevel);
+      return true;
+    } else {
+      // Try to remove the block from all queues if the block was
+      // not found in the queue for the given priority level.
+      for (int i = 0; i < LEVEL; i++) {
+        if (i != priLevel && priorityQueues.get(i).remove(block)) {
+          NameNode.blockStateChangeLog.debug(
+              "BLOCK* NameSystem.LowRedundancyBlock.remove: Removing block" +
+                  " {} from priority queue {}", block, i);
+          return true;
+        }
+      }
+    }
+    return false;
+  }
+
+  /**
+   * Recalculate and potentially update the priority level of a block.
+   *
+   * If the block priority has changed from before an attempt is made to
+   * remove it from the block queue. Regardless of whether or not the block
+   * is in the block queue of (recalculate) priority, an attempt is made
+   * to add it to that queue. This ensures that the block will be
+   * in its expected priority queue (and only that queue) by the end of the
+   * method call.
+   * @param block a low redundancy block
+   * @param curReplicas current number of replicas of the block
+   * @param decommissionedReplicas  the number of decommissioned replicas
+   * @param curExpectedReplicas expected number of replicas of the block
+   * @param curReplicasDelta the change in the replicate count from before
+   * @param expectedReplicasDelta the change in the expected replica count
+   *        from before
+   */
+  synchronized void update(BlockInfo block, int curReplicas,
+                           int readOnlyReplicas, int decommissionedReplicas,
+                           int curExpectedReplicas,
+                           int curReplicasDelta, int expectedReplicasDelta) {
+    int oldReplicas = curReplicas-curReplicasDelta;
+    int oldExpectedReplicas = curExpectedReplicas-expectedReplicasDelta;
+    int curPri = getPriority(block, curReplicas, readOnlyReplicas,
+        decommissionedReplicas, curExpectedReplicas);
+    int oldPri = getPriority(block, oldReplicas, readOnlyReplicas,
+        decommissionedReplicas, oldExpectedReplicas);
+    if(NameNode.stateChangeLog.isDebugEnabled()) {
+      NameNode.stateChangeLog.debug("LowRedundancyBlocks.update " +
+        block +
+        " curReplicas " + curReplicas +
+        " curExpectedReplicas " + curExpectedReplicas +
+        " oldReplicas " + oldReplicas +
+        " oldExpectedReplicas  " + oldExpectedReplicas +
+        " curPri  " + curPri +
+        " oldPri  " + oldPri);
+    }
+    if(oldPri != curPri) {
+      remove(block, oldPri);
+    }
+    if(priorityQueues.get(curPri).add(block)) {
+      NameNode.blockStateChangeLog.debug(
+          "BLOCK* NameSystem.LowRedundancyBlock.update: {} has only {} "
+              + "replicas and needs {} replicas so is added to "
+              + "neededReconstructions at priority level {}",
+          block, curReplicas, curExpectedReplicas, curPri);
+
+    }
+    if (oldPri != curPri || expectedReplicasDelta != 0) {
+      // corruptReplOneBlocks could possibly change
+      if (curPri == QUEUE_WITH_CORRUPT_BLOCKS &&
+          curExpectedReplicas == 1) {
+        // add a new corrupt block with replication factor 1
+        corruptReplOneBlocks++;
+      } else if (oldPri == QUEUE_WITH_CORRUPT_BLOCKS &&
+          curExpectedReplicas - expectedReplicasDelta == 1) {
+        // remove an existing corrupt block with replication factor 1
+        corruptReplOneBlocks--;
+      }
+    }
+  }
+
+  /**
+   * Get a list of block lists without sufficient redundancy. The index of
+   * block lists represents its replication priority. Iterates each block list
+   * in priority order beginning with the highest priority list. Iterators use
+   * a bookmark to resume where the previous iteration stopped. Returns when
+   * the block count is met or iteration reaches the end of the lowest priority
+   * list, in which case bookmarks for each block list are reset to the heads
+   * of their respective lists.
+   *
+   * @param blocksToProcess - number of blocks to fetch from low redundancy
+   *          blocks.
+   * @return Return a list of block lists to be replicated. The block list
+   *         index represents its redundancy priority.
+   */
+  synchronized List<List<BlockInfo>> chooseLowRedundancyBlocks(
+      int blocksToProcess) {
+    final List<List<BlockInfo>> blocksToReconstruct = new ArrayList<>(LEVEL);
+
+    int count = 0;
+    int priority = 0;
+    for (; count < blocksToProcess && priority < LEVEL; priority++) {
+      if (priority == QUEUE_WITH_CORRUPT_BLOCKS) {
+        // do not choose corrupted blocks.
+        continue;
+      }
+
+      // Go through all blocks that need reconstructions with current priority.
+      // Set the iterator to the first unprocessed block at this priority level
+      final Iterator<BlockInfo> i = priorityQueues.get(priority).getBookmark();
+      final List<BlockInfo> blocks = new LinkedList<>();
+      blocksToReconstruct.add(blocks);
+      // Loop through all remaining blocks in the list.
+      for(; count < blocksToProcess && i.hasNext(); count++) {
+        blocks.add(i.next());
+      }
+    }
+
+    if (priority == LEVEL) {
+      // Reset all bookmarks because there were no recently added blocks.
+      for (LightWeightLinkedSet<BlockInfo> q : priorityQueues) {
+        q.resetBookmark();
+      }
+    }
+
+    return blocksToReconstruct;
+  }
+
+  /** Returns an iterator of all blocks in a given priority queue. */
+  synchronized Iterator<BlockInfo> iterator(int level) {
+    return priorityQueues.get(level).iterator();
+  }
+
+  /** Return an iterator of all the low redundancy blocks. */
+  @Override
+  public synchronized Iterator<BlockInfo> iterator() {
+    final Iterator<LightWeightLinkedSet<BlockInfo>> q = priorityQueues.iterator();
+    return new Iterator<BlockInfo>() {
+      private Iterator<BlockInfo> b = q.next().iterator();
+
+      @Override
+      public BlockInfo next() {
+        hasNext();
+        return b.next();
+      }
+
+      @Override
+      public boolean hasNext() {
+        for(; !b.hasNext() && q.hasNext(); ) {
+          b = q.next().iterator();
+        }
+        return b.hasNext();
+      }
+
+      @Override
+      public void remove() {
+        throw new UnsupportedOperationException();
+      }
+    };
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/32d043d9/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/UnderReplicatedBlocks.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/UnderReplicatedBlocks.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/UnderReplicatedBlocks.java
deleted file mode 100644
index 5e8f7ed..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/UnderReplicatedBlocks.java
+++ /dev/null
@@ -1,448 +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.server.blockmanagement;
-
-import java.util.ArrayList;
-import java.util.Iterator;
-import java.util.LinkedList;
-import java.util.List;
-
-import org.apache.hadoop.hdfs.server.namenode.NameNode;
-import org.apache.hadoop.hdfs.util.LightWeightLinkedSet;
-
-/**
- * Keep prioritized queues of under replicated blocks.
- * Blocks have replication priority, with priority {@link #QUEUE_HIGHEST_PRIORITY}
- * indicating the highest priority.
- * </p>
- * Having a prioritised queue allows the {@link BlockManager} to select
- * which blocks to replicate first -it tries to give priority to data
- * that is most at risk or considered most valuable.
- *
- * <p/>
- * The policy for choosing which priority to give added blocks
- * is implemented in {@link #getPriority(BlockInfo, int, int, int, int)}.
- * </p>
- * <p>The queue order is as follows:</p>
- * <ol>
- *   <li>{@link #QUEUE_HIGHEST_PRIORITY}: the blocks that must be replicated
- *   first. That is blocks with only one copy, or blocks with zero live
- *   copies but a copy in a node being decommissioned. These blocks
- *   are at risk of loss if the disk or server on which they
- *   remain fails.</li>
- *   <li>{@link #QUEUE_VERY_UNDER_REPLICATED}: blocks that are very
- *   under-replicated compared to their expected values. Currently
- *   that means the ratio of the ratio of actual:expected means that
- *   there is <i>less than</i> 1:3.</li>. These blocks may not be at risk,
- *   but they are clearly considered "important".
- *   <li>{@link #QUEUE_UNDER_REPLICATED}: blocks that are also under
- *   replicated, and the ratio of actual:expected is good enough that
- *   they do not need to go into the {@link #QUEUE_VERY_UNDER_REPLICATED}
- *   queue.</li>
- *   <li>{@link #QUEUE_REPLICAS_BADLY_DISTRIBUTED}: there are as least as
- *   many copies of a block as required, but the blocks are not adequately
- *   distributed. Loss of a rack/switch could take all copies off-line.</li>
- *   <li>{@link #QUEUE_WITH_CORRUPT_BLOCKS} This is for blocks that are corrupt
- *   and for which there are no-non-corrupt copies (currently) available.
- *   The policy here is to keep those corrupt blocks replicated, but give
- *   blocks that are not corrupt higher priority.</li>
- * </ol>
- */
-class UnderReplicatedBlocks implements Iterable<BlockInfo> {
-  /** The total number of queues : {@value} */
-  static final int LEVEL = 5;
-  /** The queue with the highest priority: {@value} */
-  static final int QUEUE_HIGHEST_PRIORITY = 0;
-  /** The queue for blocks that are way below their expected value : {@value} */
-  static final int QUEUE_VERY_UNDER_REPLICATED = 1;
-  /** The queue for "normally" under-replicated blocks: {@value} */
-  static final int QUEUE_UNDER_REPLICATED = 2;
-  /** The queue for blocks that have the right number of replicas,
-   * but which the block manager felt were badly distributed: {@value}
-   */
-  static final int QUEUE_REPLICAS_BADLY_DISTRIBUTED = 3;
-  /** The queue for corrupt blocks: {@value} */
-  static final int QUEUE_WITH_CORRUPT_BLOCKS = 4;
-  /** the queues themselves */
-  private final List<LightWeightLinkedSet<BlockInfo>> priorityQueues
-      = new ArrayList<>(LEVEL);
-
-  /** The number of corrupt blocks with replication factor 1 */
-  private int corruptReplOneBlocks = 0;
-
-  /** Create an object. */
-  UnderReplicatedBlocks() {
-    for (int i = 0; i < LEVEL; i++) {
-      priorityQueues.add(new LightWeightLinkedSet<BlockInfo>());
-    }
-  }
-
-  /**
-   * Empty the queues.
-   */
-  synchronized void clear() {
-    for (int i = 0; i < LEVEL; i++) {
-      priorityQueues.get(i).clear();
-    }
-    corruptReplOneBlocks = 0;
-  }
-
-  /** Return the total number of under replication blocks */
-  synchronized int size() {
-    int size = 0;
-    for (int i = 0; i < LEVEL; i++) {
-      size += priorityQueues.get(i).size();
-    }
-    return size;
-  }
-
-  /** Return the number of under replication blocks excluding corrupt blocks */
-  synchronized int getUnderReplicatedBlockCount() {
-    int size = 0;
-    for (int i = 0; i < LEVEL; i++) {
-      if (i != QUEUE_WITH_CORRUPT_BLOCKS) {
-        size += priorityQueues.get(i).size();
-      }
-    }
-    return size;
-  }
-
-  /** Return the number of corrupt blocks */
-  synchronized int getCorruptBlockSize() {
-    return priorityQueues.get(QUEUE_WITH_CORRUPT_BLOCKS).size();
-  }
-
-  /** Return the number of corrupt blocks with replication factor 1 */
-  synchronized int getCorruptReplOneBlockSize() {
-    return corruptReplOneBlocks;
-  }
-
-  /** Check if a block is in the neededReplication queue */
-  synchronized boolean contains(BlockInfo block) {
-    for(LightWeightLinkedSet<BlockInfo> set : priorityQueues) {
-      if (set.contains(block)) {
-        return true;
-      }
-    }
-    return false;
-  }
-
-  /** Return the priority of a block
-   * @param curReplicas current number of replicas of the block
-   * @param expectedReplicas expected number of replicas of the block
-   * @return the priority for the blocks, between 0 and ({@link #LEVEL}-1)
-   */
-  private int getPriority(BlockInfo block,
-                          int curReplicas,
-                          int readOnlyReplicas,
-                          int decommissionedReplicas,
-                          int expectedReplicas) {
-    assert curReplicas >= 0 : "Negative replicas!";
-    if (curReplicas >= expectedReplicas) {
-      // Block has enough copies, but not enough racks
-      return QUEUE_REPLICAS_BADLY_DISTRIBUTED;
-    }
-    if (block.isStriped()) {
-      BlockInfoStriped sblk = (BlockInfoStriped) block;
-      return getPriorityStriped(curReplicas, decommissionedReplicas,
-          sblk.getRealDataBlockNum(), sblk.getParityBlockNum());
-    } else {
-      return getPriorityContiguous(curReplicas, readOnlyReplicas,
-          decommissionedReplicas, expectedReplicas);
-    }
-  }
-
-  private int getPriorityContiguous(int curReplicas, int readOnlyReplicas,
-      int decommissionedReplicas, int expectedReplicas) {
-    if (curReplicas == 0) {
-      // If there are zero non-decommissioned replicas but there are
-      // some decommissioned replicas, then assign them highest priority
-      if (decommissionedReplicas > 0) {
-        return QUEUE_HIGHEST_PRIORITY;
-      }
-      if (readOnlyReplicas > 0) {
-        // only has read-only replicas, highest risk
-        // since the read-only replicas may go down all together.
-        return QUEUE_HIGHEST_PRIORITY;
-      }
-      //all we have are corrupt blocks
-      return QUEUE_WITH_CORRUPT_BLOCKS;
-    } else if (curReplicas == 1) {
-      // only one replica, highest risk of loss
-      // highest priority
-      return QUEUE_HIGHEST_PRIORITY;
-    } else if ((curReplicas * 3) < expectedReplicas) {
-      //there is less than a third as many blocks as requested;
-      //this is considered very under-replicated
-      return QUEUE_VERY_UNDER_REPLICATED;
-    } else {
-      //add to the normal queue for under replicated blocks
-      return QUEUE_UNDER_REPLICATED;
-    }
-  }
-
-  private int getPriorityStriped(int curReplicas, int decommissionedReplicas,
-      short dataBlkNum, short parityBlkNum) {
-    if (curReplicas < dataBlkNum) {
-      // There are some replicas on decommissioned nodes so it's not corrupted
-      if (curReplicas + decommissionedReplicas >= dataBlkNum) {
-        return QUEUE_HIGHEST_PRIORITY;
-      }
-      return QUEUE_WITH_CORRUPT_BLOCKS;
-    } else if (curReplicas == dataBlkNum) {
-      // highest risk of loss, highest priority
-      return QUEUE_HIGHEST_PRIORITY;
-    } else if ((curReplicas - dataBlkNum) * 3 < parityBlkNum + 1) {
-      // there is less than a third as many blocks as requested;
-      // this is considered very under-replicated
-      return QUEUE_VERY_UNDER_REPLICATED;
-    } else {
-      // add to the normal queue for under replicated blocks
-      return QUEUE_UNDER_REPLICATED;
-    }
-  }
-
-  /** add a block to a under replication queue according to its priority
-   * @param block a under replication block
-   * @param curReplicas current number of replicas of the block
-   * @param decomissionedReplicas the number of decommissioned replicas
-   * @param expectedReplicas expected number of replicas of the block
-   * @return true if the block was added to a queue.
-   */
-  synchronized boolean add(BlockInfo block,
-                           int curReplicas,
-                           int readOnlyReplicas,
-                           int decomissionedReplicas,
-                           int expectedReplicas) {
-    assert curReplicas >= 0 : "Negative replicas!";
-    final int priLevel = getPriority(block, curReplicas, readOnlyReplicas,
-        decomissionedReplicas, expectedReplicas);
-    if(priorityQueues.get(priLevel).add(block)) {
-      if (priLevel == QUEUE_WITH_CORRUPT_BLOCKS &&
-          expectedReplicas == 1) {
-        corruptReplOneBlocks++;
-      }
-      NameNode.blockStateChangeLog.debug(
-          "BLOCK* NameSystem.UnderReplicationBlock.add: {}"
-              + " has only {} replicas and need {} replicas so is added to" +
-              " neededReplications at priority level {}", block, curReplicas,
-          expectedReplicas, priLevel);
-
-      return true;
-    }
-    return false;
-  }
-
-  /** remove a block from a under replication queue */
-  synchronized boolean remove(BlockInfo block,
-                              int oldReplicas,
-                              int oldReadOnlyReplicas,
-                              int decommissionedReplicas,
-                              int oldExpectedReplicas) {
-    final int priLevel = getPriority(block, oldReplicas, oldReadOnlyReplicas,
-        decommissionedReplicas, oldExpectedReplicas);
-    boolean removedBlock = remove(block, priLevel);
-    if (priLevel == QUEUE_WITH_CORRUPT_BLOCKS &&
-        oldExpectedReplicas == 1 &&
-        removedBlock) {
-      corruptReplOneBlocks--;
-      assert corruptReplOneBlocks >= 0 :
-          "Number of corrupt blocks with replication factor 1 " +
-              "should be non-negative";
-    }
-    return removedBlock;
-  }
-
-  /**
-   * Remove a block from the under replication queues.
-   *
-   * The priLevel parameter is a hint of which queue to query
-   * first: if negative or &gt;= {@link #LEVEL} this shortcutting
-   * is not attmpted.
-   *
-   * If the block is not found in the nominated queue, an attempt is made to
-   * remove it from all queues.
-   *
-   * <i>Warning:</i> This is not a synchronized method.
-   * @param block block to remove
-   * @param priLevel expected privilege level
-   * @return true if the block was found and removed from one of the priority queues
-   */
-  boolean remove(BlockInfo block, int priLevel) {
-    if(priLevel >= 0 && priLevel < LEVEL
-        && priorityQueues.get(priLevel).remove(block)) {
-      NameNode.blockStateChangeLog.debug(
-        "BLOCK* NameSystem.UnderReplicationBlock.remove: Removing block {}" +
-            " from priority queue {}", block, priLevel);
-      return true;
-    } else {
-      // Try to remove the block from all queues if the block was
-      // not found in the queue for the given priority level.
-      for (int i = 0; i < LEVEL; i++) {
-        if (i != priLevel && priorityQueues.get(i).remove(block)) {
-          NameNode.blockStateChangeLog.debug(
-              "BLOCK* NameSystem.UnderReplicationBlock.remove: Removing block" +
-                  " {} from priority queue {}", block, i);
-          return true;
-        }
-      }
-    }
-    return false;
-  }
-
-  /**
-   * Recalculate and potentially update the priority level of a block.
-   *
-   * If the block priority has changed from before an attempt is made to
-   * remove it from the block queue. Regardless of whether or not the block
-   * is in the block queue of (recalculate) priority, an attempt is made
-   * to add it to that queue. This ensures that the block will be
-   * in its expected priority queue (and only that queue) by the end of the
-   * method call.
-   * @param block a under replicated block
-   * @param curReplicas current number of replicas of the block
-   * @param decommissionedReplicas  the number of decommissioned replicas
-   * @param curExpectedReplicas expected number of replicas of the block
-   * @param curReplicasDelta the change in the replicate count from before
-   * @param expectedReplicasDelta the change in the expected replica count from before
-   */
-  synchronized void update(BlockInfo block, int curReplicas,
-                           int readOnlyReplicas, int decommissionedReplicas,
-                           int curExpectedReplicas,
-                           int curReplicasDelta, int expectedReplicasDelta) {
-    int oldReplicas = curReplicas-curReplicasDelta;
-    int oldExpectedReplicas = curExpectedReplicas-expectedReplicasDelta;
-    int curPri = getPriority(block, curReplicas, readOnlyReplicas,
-        decommissionedReplicas, curExpectedReplicas);
-    int oldPri = getPriority(block, oldReplicas, readOnlyReplicas,
-        decommissionedReplicas, oldExpectedReplicas);
-    if(NameNode.stateChangeLog.isDebugEnabled()) {
-      NameNode.stateChangeLog.debug("UnderReplicationBlocks.update " + 
-        block +
-        " curReplicas " + curReplicas +
-        " curExpectedReplicas " + curExpectedReplicas +
-        " oldReplicas " + oldReplicas +
-        " oldExpectedReplicas  " + oldExpectedReplicas +
-        " curPri  " + curPri +
-        " oldPri  " + oldPri);
-    }
-    if(oldPri != curPri) {
-      remove(block, oldPri);
-    }
-    if(priorityQueues.get(curPri).add(block)) {
-      NameNode.blockStateChangeLog.debug(
-          "BLOCK* NameSystem.UnderReplicationBlock.update: {} has only {} " +
-              "replicas and needs {} replicas so is added to " +
-              "neededReplications at priority level {}", block, curReplicas,
-          curExpectedReplicas, curPri);
-
-    }
-    if (oldPri != curPri || expectedReplicasDelta != 0) {
-      // corruptReplOneBlocks could possibly change
-      if (curPri == QUEUE_WITH_CORRUPT_BLOCKS &&
-          curExpectedReplicas == 1) {
-        // add a new corrupt block with replication factor 1
-        corruptReplOneBlocks++;
-      } else if (oldPri == QUEUE_WITH_CORRUPT_BLOCKS &&
-          curExpectedReplicas - expectedReplicasDelta == 1) {
-        // remove an existing corrupt block with replication factor 1
-        corruptReplOneBlocks--;
-      }
-    }
-  }
-  
-  /**
-   * Get a list of block lists to be replicated. The index of block lists
-   * represents its replication priority. Iterates each block list in priority
-   * order beginning with the highest priority list. Iterators use a bookmark to
-   * resume where the previous iteration stopped. Returns when the block count
-   * is met or iteration reaches the end of the lowest priority list, in which
-   * case bookmarks for each block list are reset to the heads of their
-   * respective lists.
-   *
-   * @param blocksToProcess - number of blocks to fetch from underReplicated
-   *          blocks.
-   * @return Return a list of block lists to be replicated. The block list index
-   *         represents its replication priority.
-   */
-  synchronized List<List<BlockInfo>> chooseUnderReplicatedBlocks(
-      int blocksToProcess) {
-    final List<List<BlockInfo>> blocksToReplicate = new ArrayList<>(LEVEL);
-    
-    int count = 0;
-    int priority = 0;
-    for (; count < blocksToProcess && priority < LEVEL; priority++) {
-      if (priority == QUEUE_WITH_CORRUPT_BLOCKS) {
-        // do not choose corrupted blocks.
-        continue;
-      }
-
-      // Go through all blocks that need replications with current priority.
-      // Set the iterator to the first unprocessed block at this priority level.
-      final Iterator<BlockInfo> i = priorityQueues.get(priority).getBookmark();
-      final List<BlockInfo> blocks = new LinkedList<>();
-      blocksToReplicate.add(blocks);
-      // Loop through all remaining blocks in the list.
-      for(; count < blocksToProcess && i.hasNext(); count++) {
-        blocks.add(i.next());
-      }
-    }
-
-    if (priority == LEVEL) {
-      // Reset all bookmarks because there were no recently added blocks.
-      for (LightWeightLinkedSet<BlockInfo> q : priorityQueues) {
-        q.resetBookmark();
-      }
-    }
-
-    return blocksToReplicate;
-  }
-
-  /** returns an iterator of all blocks in a given priority queue */
-  synchronized Iterator<BlockInfo> iterator(int level) {
-    return priorityQueues.get(level).iterator();
-  }
-
-  /** return an iterator of all the under replication blocks */
-  @Override
-  public synchronized Iterator<BlockInfo> iterator() {
-    final Iterator<LightWeightLinkedSet<BlockInfo>> q = priorityQueues.iterator();
-    return new Iterator<BlockInfo>() {
-      private Iterator<BlockInfo> b = q.next().iterator();
-
-      @Override
-      public BlockInfo next() {
-        hasNext();
-        return b.next();
-      }
-
-      @Override
-      public boolean hasNext() {
-        for(; !b.hasNext() && q.hasNext(); ) {
-          b = q.next().iterator();
-        }
-        return b.hasNext();
-      }
-
-      @Override
-      public void remove() {
-        throw new UnsupportedOperationException();
-      }
-    };
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/32d043d9/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManagerTestUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManagerTestUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManagerTestUtil.java
index c0a4fdb..1b565ee 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManagerTestUtil.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManagerTestUtil.java
@@ -74,7 +74,7 @@ public class BlockManagerTestUtil {
       final BlockInfo storedBlock = bm.getStoredBlock(b);
       return new int[]{getNumberOfRacks(bm, b),
           bm.countNodes(storedBlock).liveReplicas(),
-          bm.neededReplications.contains(storedBlock) ? 1 : 0};
+          bm.neededReconstruction.contains(storedBlock) ? 1 : 0};
     } finally {
       namesystem.readUnlock();
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/32d043d9/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockManager.java
index 5511b99..3a974e4 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockManager.java
@@ -397,20 +397,20 @@ public class TestBlockManager {
     addNodes(nodes);
     List<DatanodeDescriptor> origNodes = nodes.subList(0, 3);
     for (int i = 0; i < NUM_TEST_ITERS; i++) {
-      doTestSingleRackClusterIsSufficientlyReplicated(i, origNodes);
+      doTestSingleRackClusterHasSufficientRedundancy(i, origNodes);
     }
   }
   
-  private void doTestSingleRackClusterIsSufficientlyReplicated(int testIndex,
+  private void doTestSingleRackClusterHasSufficientRedundancy(int testIndex,
       List<DatanodeDescriptor> origNodes)
       throws Exception {
     assertEquals(0, bm.numOfUnderReplicatedBlocks());
     BlockInfo block = addBlockOnNodes(testIndex, origNodes);
-    assertFalse(bm.isNeededReplication(block, bm.countLiveNodes(block)));
+    assertFalse(bm.isNeededReconstruction(block, bm.countLiveNodes(block)));
   }
   
   @Test(timeout = 60000)
-  public void testNeededReplicationWhileAppending() throws IOException {
+  public void testNeededReconstructionWhileAppending() throws IOException {
     Configuration conf = new HdfsConfiguration();
     String src = "/test-file";
     Path file = new Path(src);
@@ -449,7 +449,7 @@ public class TestBlockManager {
         namenode.updatePipeline(clientName, oldBlock, newBlock,
             newLocatedBlock.getLocations(), newLocatedBlock.getStorageIDs());
         BlockInfo bi = bm.getStoredBlock(newBlock.getLocalBlock());
-        assertFalse(bm.isNeededReplication(bi, bm.countLiveNodes(bi)));
+        assertFalse(bm.isNeededReconstruction(bi, bm.countLiveNodes(bi)));
       } finally {
         IOUtils.closeStream(out);
       }
@@ -601,7 +601,7 @@ public class TestBlockManager {
             liveNodes,
             new NumberReplicas(),
             new ArrayList<Byte>(),
-            UnderReplicatedBlocks.QUEUE_HIGHEST_PRIORITY)[0]);
+            LowRedundancyBlocks.QUEUE_HIGHEST_PRIORITY)[0]);
 
     assertEquals("Does not choose a source node for a less-than-highest-priority"
             + " replication since all available source nodes have reached"
@@ -612,7 +612,7 @@ public class TestBlockManager {
             liveNodes,
             new NumberReplicas(),
             new ArrayList<Byte>(),
-            UnderReplicatedBlocks.QUEUE_VERY_UNDER_REPLICATED).length);
+            LowRedundancyBlocks.QUEUE_VERY_LOW_REDUNDANCY).length);
 
     // Increase the replication count to test replication count > hard limit
     DatanodeStorageInfo targets[] = { origNodes.get(1).getStorageInfos()[0] };
@@ -626,7 +626,7 @@ public class TestBlockManager {
             liveNodes,
             new NumberReplicas(),
             new ArrayList<Byte>(),
-            UnderReplicatedBlocks.QUEUE_HIGHEST_PRIORITY).length);
+            LowRedundancyBlocks.QUEUE_HIGHEST_PRIORITY).length);
   }
 
   @Test
@@ -652,7 +652,7 @@ public class TestBlockManager {
             cntNodes,
             liveNodes,
             new NumberReplicas(), new LinkedList<Byte>(),
-            UnderReplicatedBlocks.QUEUE_UNDER_REPLICATED)[0]);
+            LowRedundancyBlocks.QUEUE_LOW_REDUNDANCY)[0]);
 
 
     // Increase the replication count to test replication count > hard limit
@@ -666,7 +666,7 @@ public class TestBlockManager {
             cntNodes,
             liveNodes,
             new NumberReplicas(), new LinkedList<Byte>(),
-            UnderReplicatedBlocks.QUEUE_UNDER_REPLICATED).length);
+            LowRedundancyBlocks.QUEUE_LOW_REDUNDANCY).length);
   }
 
   @Test

http://git-wip-us.apache.org/repos/asf/hadoop/blob/32d043d9/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestLowRedundancyBlockQueues.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestLowRedundancyBlockQueues.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestLowRedundancyBlockQueues.java
new file mode 100644
index 0000000..2eb7abf
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestLowRedundancyBlockQueues.java
@@ -0,0 +1,182 @@
+/**
+ * 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 java.util.Iterator;
+
+import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.server.namenode.ErasureCodingPolicyManager;
+import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.fail;
+
+public class TestLowRedundancyBlockQueues {
+
+  private final ErasureCodingPolicy ecPolicy =
+      ErasureCodingPolicyManager.getSystemDefaultPolicy();
+
+  private BlockInfo genBlockInfo(long id) {
+    return new BlockInfoContiguous(new Block(id), (short) 3);
+  }
+
+  private BlockInfo genStripedBlockInfo(long id, long numBytes) {
+    BlockInfoStriped sblk =  new BlockInfoStriped(new Block(id), ecPolicy);
+    sblk.setNumBytes(numBytes);
+    return sblk;
+  }
+
+  /**
+   * Test that adding blocks with different replication counts puts them
+   * into different queues
+   * @throws Throwable if something goes wrong
+   */
+  @Test
+  public void testBlockPriorities() throws Throwable {
+    LowRedundancyBlocks queues = new LowRedundancyBlocks();
+    BlockInfo block1 = genBlockInfo(1);
+    BlockInfo block2 = genBlockInfo(2);
+    BlockInfo block_very_low_redundancy = genBlockInfo(3);
+    BlockInfo block_corrupt = genBlockInfo(4);
+    BlockInfo block_corrupt_repl_one = genBlockInfo(5);
+
+    //add a block with a single entry
+    assertAdded(queues, block1, 1, 0, 3);
+
+    assertEquals(1, queues.getLowRedundancyBlockCount());
+    assertEquals(1, queues.size());
+    assertInLevel(queues, block1, LowRedundancyBlocks.QUEUE_HIGHEST_PRIORITY);
+    //repeated additions fail
+    assertFalse(queues.add(block1, 1, 0, 0, 3));
+
+    //add a second block with two replicas
+    assertAdded(queues, block2, 2, 0, 3);
+    assertEquals(2, queues.getLowRedundancyBlockCount());
+    assertEquals(2, queues.size());
+    assertInLevel(queues, block2, LowRedundancyBlocks.QUEUE_LOW_REDUNDANCY);
+    //now try to add a block that is corrupt
+    assertAdded(queues, block_corrupt, 0, 0, 3);
+    assertEquals(3, queues.size());
+    assertEquals(2, queues.getLowRedundancyBlockCount());
+    assertEquals(1, queues.getCorruptBlockSize());
+    assertInLevel(queues, block_corrupt,
+                  LowRedundancyBlocks.QUEUE_WITH_CORRUPT_BLOCKS);
+
+    //insert a very insufficiently redundancy block
+    assertAdded(queues, block_very_low_redundancy, 4, 0, 25);
+    assertInLevel(queues, block_very_low_redundancy,
+                  LowRedundancyBlocks.QUEUE_VERY_LOW_REDUNDANCY);
+
+    //insert a corrupt block with replication factor 1
+    assertAdded(queues, block_corrupt_repl_one, 0, 0, 1);
+    assertEquals(2, queues.getCorruptBlockSize());
+    assertEquals(1, queues.getCorruptReplOneBlockSize());
+    queues.update(block_corrupt_repl_one, 0, 0, 0, 3, 0, 2);
+    assertEquals(0, queues.getCorruptReplOneBlockSize());
+    queues.update(block_corrupt, 0, 0, 0, 1, 0, -2);
+    assertEquals(1, queues.getCorruptReplOneBlockSize());
+    queues.update(block_very_low_redundancy, 0, 0, 0, 1, -4, -24);
+    assertEquals(2, queues.getCorruptReplOneBlockSize());
+  }
+
+  @Test
+  public void testStripedBlockPriorities() throws Throwable {
+    int dataBlkNum = ecPolicy.getNumDataUnits();
+    int parityBlkNUm = ecPolicy.getNumParityUnits();
+    doTestStripedBlockPriorities(1, parityBlkNUm);
+    doTestStripedBlockPriorities(dataBlkNum, parityBlkNUm);
+  }
+
+  private void doTestStripedBlockPriorities(int dataBlkNum, int parityBlkNum)
+      throws Throwable {
+    int groupSize = dataBlkNum + parityBlkNum;
+    long numBytes = ecPolicy.getCellSize() * dataBlkNum;
+    LowRedundancyBlocks queues = new LowRedundancyBlocks();
+    int numUR = 0;
+    int numCorrupt = 0;
+
+    // add low redundancy blocks
+    for (int i = 0; dataBlkNum + i < groupSize; i++) {
+      BlockInfo block = genStripedBlockInfo(-100 - 100 * i, numBytes);
+      assertAdded(queues, block, dataBlkNum + i, 0, groupSize);
+      numUR++;
+      assertEquals(numUR, queues.getLowRedundancyBlockCount());
+      assertEquals(numUR + numCorrupt, queues.size());
+      if (i == 0) {
+        assertInLevel(queues, block,
+            LowRedundancyBlocks.QUEUE_HIGHEST_PRIORITY);
+      } else if (i * 3 < parityBlkNum + 1) {
+        assertInLevel(queues, block,
+            LowRedundancyBlocks.QUEUE_VERY_LOW_REDUNDANCY);
+      } else {
+        assertInLevel(queues, block,
+            LowRedundancyBlocks.QUEUE_LOW_REDUNDANCY);
+      }
+    }
+
+    // add a corrupted block
+    BlockInfo block_corrupt = genStripedBlockInfo(-10, numBytes);
+    assertEquals(numCorrupt, queues.getCorruptBlockSize());
+    assertAdded(queues, block_corrupt, dataBlkNum - 1, 0, groupSize);
+    numCorrupt++;
+    assertEquals(numUR + numCorrupt, queues.size());
+    assertEquals(numUR, queues.getLowRedundancyBlockCount());
+    assertEquals(numCorrupt, queues.getCorruptBlockSize());
+    assertInLevel(queues, block_corrupt,
+        LowRedundancyBlocks.QUEUE_WITH_CORRUPT_BLOCKS);
+  }
+
+  private void assertAdded(LowRedundancyBlocks queues,
+                           BlockInfo block,
+                           int curReplicas,
+                           int decomissionedReplicas,
+                           int expectedReplicas) {
+    assertTrue("Failed to add " + block,
+               queues.add(block,
+                          curReplicas, 0,
+                          decomissionedReplicas,
+                          expectedReplicas));
+  }
+
+  /**
+   * Determine whether or not a block is in a level without changing the API.
+   * Instead get the per-level iterator and run though it looking for a match.
+   * If the block is not found, an assertion is thrown.
+   *
+   * This is inefficient, but this is only a test case.
+   * @param queues queues to scan
+   * @param block block to look for
+   * @param level level to select
+   */
+  private void assertInLevel(LowRedundancyBlocks queues,
+                             Block block,
+                             int level) {
+    final Iterator<BlockInfo> bi = queues.iterator(level);
+    while (bi.hasNext()) {
+      Block next = bi.next();
+      if (block.equals(next)) {
+        return;
+      }
+    }
+    fail("Block " + block + " not found in level " + level);
+  }
+}


[37/46] hadoop git commit: YARN-4108. CapacityScheduler: Improve preemption to only kill containers that would satisfy the incoming request. (Wangda Tan)

Posted by ae...@apache.org.
YARN-4108. CapacityScheduler: Improve preemption to only kill containers that would satisfy the incoming request. (Wangda Tan)

(cherry picked from commit 7e8c9beb4156dcaeb3a11e60aaa06d2370626913)


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

Branch: refs/heads/HDFS-7240
Commit: ae14e5d07f1b6702a5160637438028bb03d9387e
Parents: fa7a435
Author: Wangda Tan <wa...@apache.org>
Authored: Wed Mar 16 16:59:59 2016 -0700
Committer: Wangda Tan <wa...@apache.org>
Committed: Wed Mar 16 17:02:33 2016 -0700

----------------------------------------------------------------------
 .../ProportionalCapacityPreemptionPolicy.java   | 166 +++--
 .../rmcontainer/RMContainer.java                |   1 +
 .../scheduler/PreemptableResourceScheduler.java |   2 +-
 .../scheduler/ResourceLimits.java               |   9 +
 .../scheduler/SchedulerNode.java                |   9 +-
 .../scheduler/capacity/AbstractCSQueue.java     |  45 +-
 .../scheduler/capacity/CSAssignment.java        |  11 +
 .../scheduler/capacity/CapacityScheduler.java   | 132 +++-
 .../CapacitySchedulerConfiguration.java         |  14 +-
 .../capacity/CapacitySchedulerContext.java      |  15 +-
 .../scheduler/capacity/LeafQueue.java           |  69 ++
 .../scheduler/capacity/ParentQueue.java         | 157 ++++-
 .../allocator/AbstractContainerAllocator.java   |   2 +
 .../capacity/allocator/ContainerAllocation.java |  12 +
 .../allocator/RegularContainerAllocator.java    |  39 +-
 .../capacity/preemption/KillableContainer.java  |  45 ++
 .../capacity/preemption/PreemptableQueue.java   | 102 +++
 .../capacity/preemption/PreemptionManager.java  | 165 +++++
 .../scheduler/common/AssignmentInformation.java |   6 +-
 .../scheduler/common/fica/FiCaSchedulerApp.java |  32 +-
 .../common/fica/FiCaSchedulerNode.java          |  65 +-
 .../scheduler/event/SchedulerEventType.java     |  15 +-
 .../resourcemanager/TestRMDispatcher.java       |   4 +-
 .../server/resourcemanager/TestRMRestart.java   |   2 +-
 .../applicationsmanager/TestAMRestart.java      |   7 +-
 ...estProportionalCapacityPreemptionPolicy.java |   6 +-
 ...pacityPreemptionPolicyForNodePartitions.java |   2 +
 .../capacity/TestApplicationLimits.java         |   2 +
 .../capacity/TestApplicationPriority.java       |   6 +-
 .../capacity/TestCapacityScheduler.java         |   8 +-
 .../TestCapacitySchedulerPreemption.java        | 677 +++++++++++++++++++
 .../scheduler/capacity/TestChildQueueOrder.java |   2 +
 .../scheduler/capacity/TestLeafQueue.java       |   3 +
 .../TestNodeLabelContainerAllocation.java       |  97 +++
 .../scheduler/capacity/TestParentQueue.java     |   2 +
 .../scheduler/capacity/TestReservations.java    |   2 +
 .../scheduler/capacity/TestUtils.java           |  36 +
 .../fair/TestFairSchedulerPreemption.java       |   2 +-
 38 files changed, 1785 insertions(+), 186 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/ae14e5d0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/ProportionalCapacityPreemptionPolicy.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/monitor/capacity/ProportionalCapacityPreemptionPolicy.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/ProportionalCapacityPreemptionPolicy.java
index 3a87edb..9b499c8 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/ProportionalCapacityPreemptionPolicy.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/ProportionalCapacityPreemptionPolicy.java
@@ -35,7 +35,7 @@ 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.ApplicationAttemptId;
-import org.apache.hadoop.yarn.api.records.NodeId;
+import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.Priority;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
@@ -49,6 +49,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.Capacity
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.LeafQueue;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueueCapacities;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.preemption.PreemptableQueue;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerApp;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.ContainerPreemptEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEventType;
@@ -125,8 +126,8 @@ public class ProportionalCapacityPreemptionPolicy implements SchedulingEditPolic
   private long maxWaitTime;
   private CapacityScheduler scheduler;
   private long monitoringInterval;
-  private final Map<RMContainer,Long> preempted =
-    new HashMap<RMContainer,Long>();
+  private final Map<RMContainer, Long> preempted = new HashMap<>();
+
   private ResourceCalculator rc;
   private float percentageClusterPreemptionAllowed;
   private double naturalTerminationFactor;
@@ -135,6 +136,10 @@ public class ProportionalCapacityPreemptionPolicy implements SchedulingEditPolic
       new HashMap<>();
   private RMNodeLabelsManager nlm;
 
+  // Preemptable Entities, synced from scheduler at every run
+  private Map<String, PreemptableQueue> preemptableEntities = null;
+  private Set<ContainerId> killableContainers;
+
   public ProportionalCapacityPreemptionPolicy() {
     clock = SystemClock.getInstance();
   }
@@ -184,6 +189,64 @@ public class ProportionalCapacityPreemptionPolicy implements SchedulingEditPolic
     Resource clusterResources = Resources.clone(scheduler.getClusterResource());
     containerBasedPreemptOrKill(root, clusterResources);
   }
+
+  @SuppressWarnings("unchecked")
+  private void cleanupStaledKillableContainers(Resource cluster,
+      Set<String> leafQueueNames) {
+    for (String q : leafQueueNames) {
+      for (TempQueuePerPartition tq : getQueuePartitions(q)) {
+        // When queue's used - killable <= guaranteed and, killable > 0, we need
+        // to check if any of killable containers needs to be reverted
+        if (Resources.lessThanOrEqual(rc, cluster,
+            Resources.subtract(tq.current, tq.killable), tq.idealAssigned)
+            && Resources.greaterThan(rc, cluster, tq.killable, Resources.none())) {
+          // How many killable resources need to be reverted
+          // need-to-revert = already-marked-killable - (current - ideal)
+          Resource toBeRevertedFromKillable = Resources.subtract(tq.killable,
+              Resources.subtract(tq.current, tq.idealAssigned));
+
+          Resource alreadyReverted = Resources.createResource(0);
+
+          for (RMContainer c : preemptableEntities.get(q).getKillableContainers(
+              tq.partition).values()) {
+            if (Resources.greaterThanOrEqual(rc, cluster, alreadyReverted,
+                toBeRevertedFromKillable)) {
+              break;
+            }
+
+            if (Resources.greaterThan(rc, cluster,
+                Resources.add(alreadyReverted, c.getAllocatedResource()),
+                toBeRevertedFromKillable)) {
+              continue;
+            } else {
+              // This container need to be marked to unkillable
+              Resources.addTo(alreadyReverted, c.getAllocatedResource());
+              rmContext.getDispatcher().getEventHandler().handle(
+                  new ContainerPreemptEvent(c.getApplicationAttemptId(), c,
+                      SchedulerEventType.MARK_CONTAINER_FOR_NONKILLABLE));
+            }
+          }
+
+        }
+      }
+    }
+  }
+
+  private void syncKillableContainersFromScheduler() {
+    // sync preemptable entities from scheduler
+    preemptableEntities =
+        scheduler.getPreemptionManager().getShallowCopyOfPreemptableEntities();
+
+    killableContainers = new HashSet<>();
+    for (Map.Entry<String, PreemptableQueue> entry : preemptableEntities
+        .entrySet()) {
+      PreemptableQueue entity = entry.getValue();
+      for (Map<ContainerId, RMContainer> map : entity.getKillableContainers()
+          .values()) {
+        killableContainers.addAll(map.keySet());
+      }
+    }
+  }
   
   /**
    * This method selects and tracks containers to be preempted. If a container
@@ -201,6 +264,8 @@ public class ProportionalCapacityPreemptionPolicy implements SchedulingEditPolic
         .getNodeLabelManager().getClusterNodeLabelNames());
     allPartitions.add(RMNodeLabelsManager.NO_LABEL);
 
+    syncKillableContainersFromScheduler();
+
     // extract a summary of the queues from scheduler
     synchronized (scheduler) {
       queueToPartitions.clear();
@@ -228,13 +293,17 @@ public class ProportionalCapacityPreemptionPolicy implements SchedulingEditPolic
           recursivelyComputeIdealAssignment(tRoot, totalPreemptionAllowed);
     }
 
+    // remove containers from killable list when we want to preempt less resources
+    // from queue.
+    cleanupStaledKillableContainers(clusterResources, leafQueueNames);
+
     // based on ideal allocation select containers to be preempted from each
     // queue and each application
     Map<ApplicationAttemptId,Set<RMContainer>> toPreempt =
         getContainersToPreempt(leafQueueNames, clusterResources);
 
     if (LOG.isDebugEnabled()) {
-      logToCSV(new ArrayList<String>(leafQueueNames));
+      logToCSV(new ArrayList<>(leafQueueNames));
     }
 
     // if we are in observeOnly mode return before any action is taken
@@ -254,10 +323,10 @@ public class ProportionalCapacityPreemptionPolicy implements SchedulingEditPolic
         // if we tried to preempt this for more than maxWaitTime
         if (preempted.get(container) != null &&
             preempted.get(container) + maxWaitTime < clock.getTime()) {
-          // kill it
+          // mark container killable
           rmContext.getDispatcher().getEventHandler().handle(
               new ContainerPreemptEvent(appAttemptId, container,
-                  SchedulerEventType.KILL_PREEMPTED_CONTAINER));
+                  SchedulerEventType.MARK_CONTAINER_FOR_KILLABLE));
           preempted.remove(container);
         } else {
           if (preempted.get(container) != null) {
@@ -333,14 +402,14 @@ public class ProportionalCapacityPreemptionPolicy implements SchedulingEditPolic
 
     // qAlloc tracks currently active queues (will decrease progressively as
     // demand is met)
-    List<TempQueuePerPartition> qAlloc = new ArrayList<TempQueuePerPartition>(queues);
+    List<TempQueuePerPartition> qAlloc = new ArrayList<>(queues);
     // unassigned tracks how much resources are still to assign, initialized
     // with the total capacity for this set of queues
     Resource unassigned = Resources.clone(tot_guarant);
 
     // group queues based on whether they have non-zero guaranteed capacity
-    Set<TempQueuePerPartition> nonZeroGuarQueues = new HashSet<TempQueuePerPartition>();
-    Set<TempQueuePerPartition> zeroGuarQueues = new HashSet<TempQueuePerPartition>();
+    Set<TempQueuePerPartition> nonZeroGuarQueues = new HashSet<>();
+    Set<TempQueuePerPartition> zeroGuarQueues = new HashSet<>();
 
     for (TempQueuePerPartition q : qAlloc) {
       if (Resources
@@ -415,8 +484,8 @@ public class ProportionalCapacityPreemptionPolicy implements SchedulingEditPolic
     // idealAssigned >= current + pending), remove it from consideration.
     // Sort queues from most under-guaranteed to most over-guaranteed.
     TQComparator tqComparator = new TQComparator(rc, tot_guarant);
-    PriorityQueue<TempQueuePerPartition> orderedByNeed =
-        new PriorityQueue<TempQueuePerPartition>(10, tqComparator);
+    PriorityQueue<TempQueuePerPartition> orderedByNeed = new PriorityQueue<>(10,
+        tqComparator);
     for (Iterator<TempQueuePerPartition> i = qAlloc.iterator(); i.hasNext();) {
       TempQueuePerPartition q = i.next();
       if (Resources.greaterThan(rc, tot_guarant, q.current, q.guaranteed)) {
@@ -474,7 +543,7 @@ public class ProportionalCapacityPreemptionPolicy implements SchedulingEditPolic
   // percentage of guaranteed.
   protected Collection<TempQueuePerPartition> getMostUnderservedQueues(
       PriorityQueue<TempQueuePerPartition> orderedByNeed, TQComparator tqComparator) {
-    ArrayList<TempQueuePerPartition> underserved = new ArrayList<TempQueuePerPartition>();
+    ArrayList<TempQueuePerPartition> underserved = new ArrayList<>();
     while (!orderedByNeed.isEmpty()) {
       TempQueuePerPartition q1 = orderedByNeed.remove();
       underserved.add(q1);
@@ -502,7 +571,7 @@ public class ProportionalCapacityPreemptionPolicy implements SchedulingEditPolic
     
     if (ignoreGuar) {
       for (TempQueuePerPartition q : queues) {
-        q.normalizedGuarantee = (float)  1.0f / ((float) queues.size());
+        q.normalizedGuarantee = 1.0f / queues.size();
       }
     } else {
       for (TempQueuePerPartition q : queues) {
@@ -515,8 +584,9 @@ public class ProportionalCapacityPreemptionPolicy implements SchedulingEditPolic
     }
   }
 
-  private String getPartitionByNodeId(NodeId nodeId) {
-    return scheduler.getSchedulerNode(nodeId).getPartition();
+  private String getPartitionByRMContainer(RMContainer rmContainer) {
+    return scheduler.getSchedulerNode(rmContainer.getAllocatedNode())
+        .getPartition();
   }
 
   /**
@@ -534,7 +604,7 @@ public class ProportionalCapacityPreemptionPolicy implements SchedulingEditPolic
       return false;
     }
 
-    String nodePartition = getPartitionByNodeId(rmContainer.getAllocatedNode());
+    String nodePartition = getPartitionByRMContainer(rmContainer);
     Resource toObtainByPartition =
         resourceToObtainByPartitions.get(nodePartition);
 
@@ -575,7 +645,7 @@ public class ProportionalCapacityPreemptionPolicy implements SchedulingEditPolic
       ApplicationAttemptId appAttemptId, RMContainer containerToPreempt) {
     Set<RMContainer> set;
     if (null == (set = preemptMap.get(appAttemptId))) {
-      set = new HashSet<RMContainer>();
+      set = new HashSet<>();
       preemptMap.put(appAttemptId, set);
     }
     set.add(containerToPreempt);
@@ -587,7 +657,7 @@ public class ProportionalCapacityPreemptionPolicy implements SchedulingEditPolic
    * over-capacity queue. It uses {@link #NATURAL_TERMINATION_FACTOR} to
    * account for containers that will naturally complete.
    *
-   * @param queues set of leaf queues to preempt from
+   * @param leafQueueNames set of leaf queues to preempt from
    * @param clusterResource total amount of cluster resources
    * @return a map of applciationID to set of containers to preempt
    */
@@ -595,8 +665,8 @@ public class ProportionalCapacityPreemptionPolicy implements SchedulingEditPolic
       Set<String> leafQueueNames, Resource clusterResource) {
 
     Map<ApplicationAttemptId, Set<RMContainer>> preemptMap =
-        new HashMap<ApplicationAttemptId, Set<RMContainer>>();
-    List<RMContainer> skippedAMContainerlist = new ArrayList<RMContainer>();
+        new HashMap<>();
+    List<RMContainer> skippedAMContainerlist = new ArrayList<>();
 
     // Loop all leaf queues
     for (String queueName : leafQueueNames) {
@@ -614,7 +684,7 @@ public class ProportionalCapacityPreemptionPolicy implements SchedulingEditPolic
       LeafQueue leafQueue = null;
 
       Map<String, Resource> resToObtainByPartition =
-          new HashMap<String, Resource>();
+          new HashMap<>();
       for (TempQueuePerPartition qT : getQueuePartitions(queueName)) {
         leafQueue = qT.leafQueue;
         // we act only if we are violating balance by more than
@@ -703,7 +773,6 @@ public class ProportionalCapacityPreemptionPolicy implements SchedulingEditPolic
    * @param clusterResource
    * @param preemptMap
    * @param skippedAMContainerlist
-   * @param resToObtain
    * @param skippedAMSize
    * @param maxAMCapacityForThisQueue
    */
@@ -751,7 +820,7 @@ public class ProportionalCapacityPreemptionPolicy implements SchedulingEditPolic
 
     // first drop reserved containers towards rsrcPreempt
     List<RMContainer> reservedContainers =
-        new ArrayList<RMContainer>(app.getReservedContainers());
+        new ArrayList<>(app.getReservedContainers());
     for (RMContainer c : reservedContainers) {
       if (resToObtainByPartition.isEmpty()) {
         return;
@@ -771,8 +840,7 @@ public class ProportionalCapacityPreemptionPolicy implements SchedulingEditPolic
     // if more resources are to be freed go through all live containers in
     // reverse priority and reverse allocation order and mark them for
     // preemption
-    List<RMContainer> liveContainers =
-      new ArrayList<RMContainer>(app.getLiveContainers());
+    List<RMContainer> liveContainers = new ArrayList<>(app.getLiveContainers());
 
     sortContainers(liveContainers);
 
@@ -788,6 +856,11 @@ public class ProportionalCapacityPreemptionPolicy implements SchedulingEditPolic
         continue;
       }
 
+      // Skip already marked to killable containers
+      if (killableContainers.contains(c.getContainerId())) {
+        continue;
+      }
+
       // Try to preempt this container
       tryPreemptContainerAndDeductResToObtain(resToObtainByPartition, c,
           clusterResource, preemptMap);
@@ -826,6 +899,10 @@ public class ProportionalCapacityPreemptionPolicy implements SchedulingEditPolic
     return "ProportionalCapacityPreemptionPolicy";
   }
 
+  @VisibleForTesting
+  public Map<RMContainer, Long> getToPreemptContainers() {
+    return preempted;
+  }
 
   /**
    * This method walks a tree of CSQueue and clones the portion of the state
@@ -851,6 +928,11 @@ public class ProportionalCapacityPreemptionPolicy implements SchedulingEditPolic
           partitionToLookAt);
       Resource guaranteed = Resources.multiply(partitionResource, absCap);
       Resource maxCapacity = Resources.multiply(partitionResource, absMaxCap);
+      Resource killable = Resources.none();
+      if (null != preemptableEntities.get(queueName)) {
+         killable = preemptableEntities.get(queueName)
+            .getKillableResource(partitionToLookAt);
+      }
 
       // when partition is a non-exclusive partition, the actual maxCapacity
       // could more than specified maxCapacity
@@ -875,7 +957,7 @@ public class ProportionalCapacityPreemptionPolicy implements SchedulingEditPolic
               l.getTotalPendingResourcesConsideringUserLimit(
                   partitionResource, partitionToLookAt);
         ret = new TempQueuePerPartition(queueName, current, pending, guaranteed,
-            maxCapacity, preemptionDisabled, partitionToLookAt);
+            maxCapacity, preemptionDisabled, partitionToLookAt, killable);
         if (preemptionDisabled) {
           ret.untouchableExtra = extra;
         } else {
@@ -886,7 +968,7 @@ public class ProportionalCapacityPreemptionPolicy implements SchedulingEditPolic
         Resource pending = Resource.newInstance(0, 0);
         ret =
             new TempQueuePerPartition(curQueue.getQueueName(), current, pending,
-                guaranteed, maxCapacity, false, partitionToLookAt);
+                guaranteed, maxCapacity, false, partitionToLookAt, killable);
         Resource childrensPreemptable = Resource.newInstance(0, 0);
         for (CSQueue c : curQueue.getChildQueues()) {
           TempQueuePerPartition subq =
@@ -932,7 +1014,7 @@ public class ProportionalCapacityPreemptionPolicy implements SchedulingEditPolic
 
     Map<String, TempQueuePerPartition> queuePartitions;
     if (null == (queuePartitions = queueToPartitions.get(queueName))) {
-      queuePartitions = new HashMap<String, TempQueuePerPartition>();
+      queuePartitions = new HashMap<>();
       queueToPartitions.put(queueName, queuePartitions);
     }
     queuePartitions.put(queuePartition.partition, queuePartition);
@@ -971,8 +1053,10 @@ public class ProportionalCapacityPreemptionPolicy implements SchedulingEditPolic
     final Resource guaranteed;
     final Resource maxCapacity;
     final String partition;
+    final Resource killable;
     Resource idealAssigned;
     Resource toBePreempted;
+
     // For logging purpose
     Resource actuallyPreempted;
     Resource untouchableExtra;
@@ -986,7 +1070,7 @@ public class ProportionalCapacityPreemptionPolicy implements SchedulingEditPolic
 
     TempQueuePerPartition(String queueName, Resource current, Resource pending,
         Resource guaranteed, Resource maxCapacity, boolean preemptionDisabled,
-        String partition) {
+        String partition, Resource killableResource) {
       this.queueName = queueName;
       this.current = current;
       this.pending = pending;
@@ -996,11 +1080,12 @@ public class ProportionalCapacityPreemptionPolicy implements SchedulingEditPolic
       this.actuallyPreempted = Resource.newInstance(0, 0);
       this.toBePreempted = Resource.newInstance(0, 0);
       this.normalizedGuarantee = Float.NaN;
-      this.children = new ArrayList<TempQueuePerPartition>();
+      this.children = new ArrayList<>();
       this.untouchableExtra = Resource.newInstance(0, 0);
       this.preemptableExtra = Resource.newInstance(0, 0);
       this.preemptionDisabled = preemptionDisabled;
       this.partition = partition;
+      this.killable = killableResource;
     }
 
     public void setLeafQueue(LeafQueue l){
@@ -1018,12 +1103,6 @@ public class ProportionalCapacityPreemptionPolicy implements SchedulingEditPolic
       Resources.addTo(pending, q.pending);
     }
 
-    public void addChildren(ArrayList<TempQueuePerPartition> queues) {
-      assert leafQueue == null;
-      children.addAll(queues);
-    }
-
-
     public ArrayList<TempQueuePerPartition> getChildren(){
       return children;
     }
@@ -1064,18 +1143,13 @@ public class ProportionalCapacityPreemptionPolicy implements SchedulingEditPolic
       return sb.toString();
     }
 
-    public void printAll() {
-      LOG.info(this.toString());
-      for (TempQueuePerPartition sub : this.getChildren()) {
-        sub.printAll();
-      }
-    }
-
     public void assignPreemption(float scalingFactor,
         ResourceCalculator rc, Resource clusterResource) {
-      if (Resources.greaterThan(rc, clusterResource, current, idealAssigned)) {
-          toBePreempted = Resources.multiply(
-              Resources.subtract(current, idealAssigned), scalingFactor);
+      if (Resources.greaterThan(rc, clusterResource,
+          Resources.subtract(current, killable), idealAssigned)) {
+        toBePreempted = Resources.multiply(Resources.subtract(
+            Resources.subtract(current, killable), idealAssigned),
+            scalingFactor);
       } else {
         toBePreempted = Resource.newInstance(0, 0);
       }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ae14e5d0/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 5d26931..dfe0886 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
@@ -30,6 +30,7 @@ import org.apache.hadoop.yarn.api.records.Priority;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.ResourceRequest;
 import org.apache.hadoop.yarn.event.EventHandler;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Queue;
 
 /**
  * Represents the ResourceManager's view of an application container. See 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ae14e5d0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/PreemptableResourceScheduler.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/PreemptableResourceScheduler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/PreemptableResourceScheduler.java
index ee7e101..b73c538 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/PreemptableResourceScheduler.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/PreemptableResourceScheduler.java
@@ -45,6 +45,6 @@ public interface PreemptableResourceScheduler extends ResourceScheduler {
    * Ask the scheduler to forcibly interrupt the container given as input
    * @param container
    */
-  void killPreemptedContainer(RMContainer container);
+  void markContainerForKillable(RMContainer container);
 
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ae14e5d0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/ResourceLimits.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/ResourceLimits.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/ResourceLimits.java
index c545e9e..721eb36 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/ResourceLimits.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/ResourceLimits.java
@@ -38,6 +38,8 @@ public class ResourceLimits {
   // containers.
   private volatile Resource headroom;
 
+  private boolean allowPreempt = false;
+
   public ResourceLimits(Resource limit) {
     this(limit, Resources.none());
   }
@@ -72,4 +74,11 @@ public class ResourceLimits {
     this.amountNeededUnreserve = amountNeededUnreserve;
   }
 
+  public boolean isAllowPreemption() {
+    return allowPreempt;
+  }
+
+  public void setIsAllowPreemption(boolean allowPreempt) {
+   this.allowPreempt = allowPreempt;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ae14e5d0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerNode.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/SchedulerNode.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerNode.java
index 33ab2f1..6c4f300 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerNode.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerNode.java
@@ -64,9 +64,8 @@ public abstract class SchedulerNode {
   private volatile ResourceUtilization nodeUtilization =
       ResourceUtilization.newInstance(0, 0, 0f);
 
-
-  /** Set of containers that are allocated containers. */
-  private final Map<ContainerId, RMContainer> launchedContainers =
+  /* set of containers that are allocated containers */
+  protected final Map<ContainerId, RMContainer> launchedContainers =
       new HashMap<>();
 
   private final RMNode rmNode;
@@ -168,7 +167,7 @@ public abstract class SchedulerNode {
    * @param deltaResource Change in the resource allocation.
    * @param increase True if the change is an increase of allocation.
    */
-  private synchronized void changeContainerResource(ContainerId containerId,
+  protected synchronized void changeContainerResource(ContainerId containerId,
       Resource deltaResource, boolean increase) {
     if (increase) {
       deductUnallocatedResource(deltaResource);
@@ -242,7 +241,7 @@ public abstract class SchedulerNode {
    * Update the resources of the node when allocating a new container.
    * @param container Container to allocate.
    */
-  private synchronized void updateResource(Container container) {
+  protected synchronized void updateResource(Container container) {
     addUnallocatedResource(container.getResource());
     --numContainers;
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ae14e5d0/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 39ca29b..955f8fa 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
@@ -21,6 +21,7 @@ package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity;
 import java.io.IOException;
 import java.util.HashMap;
 import java.util.HashSet;
+import java.util.Iterator;
 import java.util.Map;
 import java.util.Set;
 
@@ -45,6 +46,7 @@ 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.rmcontainer.RMContainer;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceLimits;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceUsage;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplicationAttempt;
@@ -440,11 +442,8 @@ public abstract class AbstractCSQueue implements CSQueue {
           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;  
+      return Resources.min(resourceCalculator, clusterResource,
+          queueMaxResource, currentResourceLimits.getLimit());
     } 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.
@@ -474,12 +473,19 @@ public abstract class AbstractCSQueue implements CSQueue {
 
     Resource nowTotalUsed = queueUsage.getUsed(nodePartition);
 
-    // Set headroom for currentResourceLimits
-    currentResourceLimits.setHeadroom(Resources.subtract(currentLimitResource,
-        nowTotalUsed));
+    // Set headroom for currentResourceLimits:
+    // When queue is a parent queue: Headroom = limit - used + killable
+    // When queue is a leaf queue: Headroom = limit - used (leaf queue cannot preempt itself)
+    Resource usedExceptKillable = nowTotalUsed;
+    if (null != getChildQueues() && !getChildQueues().isEmpty()) {
+      usedExceptKillable = Resources.subtract(nowTotalUsed,
+          getTotalKillableResource(nodePartition));
+    }
+    currentResourceLimits.setHeadroom(
+        Resources.subtract(currentLimitResource, usedExceptKillable));
 
     if (Resources.greaterThanOrEqual(resourceCalculator, clusterResource,
-        nowTotalUsed, currentLimitResource)) {
+        usedExceptKillable, currentLimitResource)) {
 
       // if reservation continous looking enabled, check to see if could we
       // potentially use this node instead of a reserved node if the application
@@ -491,7 +497,7 @@ public abstract class AbstractCSQueue implements CSQueue {
               resourceCouldBeUnreserved, Resources.none())) {
         // resource-without-reserved = used - reserved
         Resource newTotalWithoutReservedResource =
-            Resources.subtract(nowTotalUsed, resourceCouldBeUnreserved);
+            Resources.subtract(usedExceptKillable, resourceCouldBeUnreserved);
 
         // when total-used-without-reserved-resource < currentLimit, we still
         // have chance to allocate on this node by unreserving some containers
@@ -620,11 +626,10 @@ public abstract class AbstractCSQueue implements CSQueue {
     // considering all labels in cluster, only those labels which are
     // use some resource of this queue can be considered.
     Set<String> nodeLabels = new HashSet<String>();
-    if (this.getAccessibleNodeLabels() != null
-        && this.getAccessibleNodeLabels().contains(RMNodeLabelsManager.ANY)) {
-      nodeLabels.addAll(Sets.union(this.getQueueCapacities()
-          .getNodePartitionsSet(), this.getQueueResourceUsage()
-          .getNodePartitionsSet()));
+    if (this.getAccessibleNodeLabels() != null && this.getAccessibleNodeLabels()
+        .contains(RMNodeLabelsManager.ANY)) {
+      nodeLabels.addAll(Sets.union(this.getQueueCapacities().getNodePartitionsSet(),
+          this.getQueueResourceUsage().getNodePartitionsSet()));
     } else {
       nodeLabels.addAll(this.getAccessibleNodeLabels());
     }
@@ -636,4 +641,14 @@ public abstract class AbstractCSQueue implements CSQueue {
     }
     return nodeLabels;
   }
+
+  public Resource getTotalKillableResource(String partition) {
+    return csContext.getPreemptionManager().getKillableResource(queueName,
+        partition);
+  }
+
+  public Iterator<RMContainer> getKillableContainers(String partition) {
+    return csContext.getPreemptionManager().getKillableContainers(queueName,
+        partition);
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ae14e5d0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CSAssignment.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/CSAssignment.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/CSAssignment.java
index 68f6f12..6406efe 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/CSAssignment.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/CSAssignment.java
@@ -26,6 +26,8 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.Assignment
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerApp;
 import org.apache.hadoop.yarn.util.resource.Resources;
 
+import java.util.List;
+
 @Private
 @Unstable
 public class CSAssignment {
@@ -42,6 +44,7 @@ public class CSAssignment {
   private boolean fulfilledReservation;
   private final AssignmentInformation assignmentInformation;
   private boolean increaseAllocation;
+  private List<RMContainer> containersToKill;
 
   public CSAssignment(Resource resource, NodeType type) {
     this(resource, type, null, null, false, false);
@@ -147,4 +150,12 @@ public class CSAssignment {
   public void setIncreasedAllocation(boolean flag) {
     increaseAllocation = flag;
   }
+
+  public void setContainersToKill(List<RMContainer> containersToKill) {
+    this.containersToKill = containersToKill;
+  }
+
+  public List<RMContainer> getContainersToKill() {
+    return containersToKill;
+  }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ae14e5d0/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 735306a..cf5c3b5 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
@@ -108,6 +108,8 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerDynamicE
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerHealth;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNode;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerUtils;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.preemption.KillableContainer;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.preemption.PreemptionManager;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.AssignmentInformation;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.QueueEntitlement;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerApp;
@@ -148,6 +150,10 @@ public class CapacityScheduler extends
   // timeout to join when we stop this service
   protected final long THREAD_JOIN_TIMEOUT_MS = 1000;
 
+  private PreemptionManager preemptionManager = new PreemptionManager();
+
+  private volatile boolean isLazyPreemptionEnabled = false;
+
   static final Comparator<CSQueue> nonPartitionedQueueComparator =
       new Comparator<CSQueue>() {
     @Override
@@ -298,12 +304,11 @@ public class CapacityScheduler extends
     initMaximumResourceCapability(this.conf.getMaximumAllocation());
     this.calculator = this.conf.getResourceCalculator();
     this.usePortForNodeName = this.conf.getUsePortForNodeName();
-    this.applications =
-        new ConcurrentHashMap<ApplicationId,
-            SchedulerApplication<FiCaSchedulerApp>>();
+    this.applications = new ConcurrentHashMap<>();
     this.labelManager = rmContext.getNodeLabelManager();
     authorizer = YarnAuthorizationProvider.getInstance(yarnConf);
     initializeQueues(this.conf);
+    this.isLazyPreemptionEnabled = conf.getLazyPreemptionEnabled();
 
     scheduleAsynchronously = this.conf.getScheduleAynschronously();
     asyncScheduleInterval =
@@ -369,6 +374,9 @@ public class CapacityScheduler extends
       refreshMaximumAllocation(this.conf.getMaximumAllocation());
       throw new IOException("Failed to re-init queues", t);
     }
+
+    // update lazy preemption
+    this.isLazyPreemptionEnabled = this.conf.getLazyPreemptionEnabled();
   }
   
   long getAsyncScheduleInterval() {
@@ -503,6 +511,9 @@ public class CapacityScheduler extends
     LOG.info("Initialized root queue " + root);
     updatePlacementRules();
     setQueueAcls(authorizer, queues);
+
+    // Notify Preemption Manager
+    preemptionManager.refreshQueues(null, root);
   }
 
   @Lock(CapacityScheduler.class)
@@ -531,6 +542,9 @@ public class CapacityScheduler extends
 
     labelManager.reinitializeQueueLabels(getQueueToLabels());
     setQueueAcls(authorizer, queues);
+
+    // Notify Preemption Manager
+    preemptionManager.refreshQueues(null, root);
   }
 
   @VisibleForTesting
@@ -1253,8 +1267,10 @@ public class CapacityScheduler extends
 
     // Try to schedule more if there are no reservations to fulfill
     if (node.getReservedContainer() == null) {
-      if (calculator.computeAvailableContainers(node.getUnallocatedResource(),
-        minimumAllocation) > 0) {
+      if (calculator.computeAvailableContainers(Resources
+              .add(node.getUnallocatedResource(), node.getTotalKillableResources()),
+          minimumAllocation) > 0) {
+
         if (LOG.isDebugEnabled()) {
           LOG.debug("Trying to schedule on node: " + node.getNodeName() +
               ", available: " + node.getUnallocatedResource());
@@ -1263,10 +1279,8 @@ public class CapacityScheduler extends
         assignment = root.assignContainers(
             getClusterResource(),
             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, getClusterResource())),
+                node.getPartition(), getClusterResource())),
             SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
         if (Resources.greaterThan(calculator, getClusterResource(),
             assignment.getResource(), Resources.none())) {
@@ -1436,11 +1450,20 @@ public class CapacityScheduler extends
       markContainerForPreemption(aid, containerToBePreempted);
     }
     break;
-    case KILL_PREEMPTED_CONTAINER:
+    case MARK_CONTAINER_FOR_KILLABLE:
+    {
+      ContainerPreemptEvent containerKillableEvent = (ContainerPreemptEvent)event;
+      RMContainer killableContainer = containerKillableEvent.getContainer();
+      markContainerForKillable(killableContainer);
+    }
+    break;
+    case MARK_CONTAINER_FOR_NONKILLABLE:
     {
-      ContainerPreemptEvent killContainerEvent = (ContainerPreemptEvent)event;
-      RMContainer containerToBeKilled = killContainerEvent.getContainer();
-      killPreemptedContainer(containerToBeKilled);
+      if (isLazyPreemptionEnabled) {
+        ContainerPreemptEvent cancelKillContainerEvent =
+            (ContainerPreemptEvent) event;
+        markContainerForNonKillable(cancelKillContainerEvent.getContainer());
+      }
     }
     break;
     default:
@@ -1548,14 +1571,14 @@ public class CapacityScheduler extends
   protected void completedContainerInternal(
       RMContainer rmContainer, ContainerStatus containerStatus,
       RMContainerEventType event) {
-    
     Container container = rmContainer.getContainer();
+    ContainerId containerId = container.getId();
     
     // Get the application for the finished container
     FiCaSchedulerApp application =
         getCurrentAttemptForContainer(container.getId());
     ApplicationId appId =
-        container.getId().getApplicationAttemptId().getApplicationId();
+        containerId.getApplicationAttemptId().getApplicationId();
     if (application == null) {
       LOG.info("Container " + container + " of" + " finished application "
           + appId + " completed with event " + event);
@@ -1569,15 +1592,6 @@ public class CapacityScheduler extends
     LeafQueue queue = (LeafQueue)application.getQueue();
     queue.completedContainer(getClusterResource(), application, node,
         rmContainer, containerStatus, event, null, true);
-
-    if (containerStatus.getExitStatus() == ContainerExitStatus.PREEMPTED) {
-      schedulerHealth.updatePreemption(Time.now(), container.getNodeId(),
-        container.getId(), queue.getQueuePath());
-      schedulerHealth.updateSchedulerPreemptionCounts(1);
-    } else {
-      schedulerHealth.updateRelease(lastNodeUpdateTime, container.getNodeId(),
-        container.getId(), queue.getQueuePath());
-    }
   }
   
   @Override
@@ -1613,7 +1627,7 @@ public class CapacityScheduler extends
       ApplicationAttemptId applicationAttemptId) {
     return super.getApplicationAttempt(applicationAttemptId);
   }
-  
+
   @Lock(Lock.NoLock.class)
   public FiCaSchedulerNode getNode(NodeId nodeId) {
     return nodeTracker.getNode(nodeId);
@@ -1654,15 +1668,60 @@ public class CapacityScheduler extends
     }
   }
 
-  @Override
-  public void killPreemptedContainer(RMContainer cont) {
+  public synchronized void markContainerForKillable(
+      RMContainer killableContainer) {
     if (LOG.isDebugEnabled()) {
-      LOG.debug(SchedulerEventType.KILL_PREEMPTED_CONTAINER + ": container"
-          + cont.toString());
+      LOG.debug(SchedulerEventType.MARK_CONTAINER_FOR_KILLABLE + ": container"
+          + killableContainer.toString());
+    }
+
+    if (!isLazyPreemptionEnabled) {
+      super.completedContainer(killableContainer, SchedulerUtils
+          .createPreemptedContainerStatus(killableContainer.getContainerId(),
+              SchedulerUtils.PREEMPTED_CONTAINER), RMContainerEventType.KILL);
+    } else {
+      FiCaSchedulerNode node = (FiCaSchedulerNode) getSchedulerNode(
+          killableContainer.getAllocatedNode());
+
+      FiCaSchedulerApp application = getCurrentAttemptForContainer(
+          killableContainer.getContainerId());
+
+      node.markContainerToKillable(killableContainer.getContainerId());
+
+      // notify PreemptionManager
+      // Get the application for the finished container
+      if (null != application) {
+        String leafQueueName = application.getCSLeafQueue().getQueueName();
+        getPreemptionManager().addKillableContainer(
+            new KillableContainer(killableContainer, node.getPartition(),
+                leafQueueName));
+      }    }
+  }
+
+  private synchronized void markContainerForNonKillable(
+      RMContainer nonKillableContainer) {
+    if (LOG.isDebugEnabled()) {
+      LOG.debug(
+          SchedulerEventType.MARK_CONTAINER_FOR_NONKILLABLE + ": container"
+              + nonKillableContainer.toString());
+    }
+
+    FiCaSchedulerNode node = (FiCaSchedulerNode) getSchedulerNode(
+        nonKillableContainer.getAllocatedNode());
+
+    FiCaSchedulerApp application = getCurrentAttemptForContainer(
+        nonKillableContainer.getContainerId());
+
+    node.markContainerToNonKillable(nonKillableContainer.getContainerId());
+
+    // notify PreemptionManager
+    // Get the application for the finished container
+    if (null != application) {
+      String leafQueueName = application.getCSLeafQueue().getQueueName();
+      getPreemptionManager().removeKillableContainer(
+          new KillableContainer(nonKillableContainer, node.getPartition(),
+              leafQueueName));
     }
-    super.completedContainer(cont, SchedulerUtils
-        .createPreemptedContainerStatus(cont.getContainerId(),
-        SchedulerUtils.PREEMPTED_CONTAINER), RMContainerEventType.KILL);
   }
 
   @Override
@@ -1945,6 +2004,7 @@ public class CapacityScheduler extends
     return ret;
   }
 
+  @Override
   public SchedulerHealth getSchedulerHealth() {
     return this.schedulerHealth;
   }
@@ -1954,6 +2014,11 @@ public class CapacityScheduler extends
   }
 
   @Override
+  public long getLastNodeUpdateTime() {
+    return lastNodeUpdateTime;
+  }
+
+  @Override
   public Priority checkAndGetApplicationPriority(Priority priorityFromContext,
       String user, String queueName, ApplicationId applicationId)
       throws YarnException {
@@ -2054,4 +2119,9 @@ public class CapacityScheduler extends
         + rmApp.getQueue() + " for application: " + applicationId
         + " for the user: " + rmApp.getUser());
   }
+
+  @Override
+  public PreemptionManager getPreemptionManager() {
+    return preemptionManager;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ae14e5d0/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 3756d9e..3729264 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
@@ -257,6 +257,12 @@ public class CapacitySchedulerConfiguration extends ReservationSchedulerConfigur
   public static final String RESERVATION_ENFORCEMENT_WINDOW =
       "reservation-enforcement-window";
 
+  @Private
+  public static final String LAZY_PREEMPTION_ENALBED = PREFIX + "lazy-preemption-enabled";
+
+  @Private
+  public static final boolean DEFAULT_LAZY_PREEMPTION_ENABLED = false;
+
   public CapacitySchedulerConfiguration() {
     this(new Configuration());
   }
@@ -1007,7 +1013,11 @@ public class CapacitySchedulerConfiguration extends ReservationSchedulerConfigur
   @VisibleForTesting
   public void setOrderingPolicyParameter(String queue,
       String parameterKey, String parameterValue) {
-    set(getQueuePrefix(queue) + ORDERING_POLICY + "."
-        + parameterKey, parameterValue);
+    set(getQueuePrefix(queue) + ORDERING_POLICY + "." + parameterKey,
+        parameterValue);
+  }
+
+  public boolean getLazyPreemptionEnabled() {
+    return getBoolean(LAZY_PREEMPTION_ENALBED, DEFAULT_LAZY_PREEMPTION_ENABLED);
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ae14e5d0/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 2a0dd0d..1203272 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
@@ -18,17 +18,20 @@
 
 package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity;
 
-import java.util.Comparator;
-
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerHealth;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.preemption.PreemptionManager;
 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.server.resourcemanager.security.RMContainerTokenSecretManager;
 import org.apache.hadoop.yarn.util.resource.ResourceCalculator;
 
+import java.util.Comparator;
+
 /**
  * Read-only interface to {@link CapacityScheduler} context.
  */
@@ -61,4 +64,12 @@ public interface CapacitySchedulerContext {
   PartitionedQueueComparator getPartitionedQueueComparator();
   
   FiCaSchedulerNode getNode(NodeId nodeId);
+
+  FiCaSchedulerApp getApplicationAttempt(ApplicationAttemptId attemptId);
+
+  PreemptionManager getPreemptionManager();
+
+  SchedulerHealth getSchedulerHealth();
+
+  long getLastNodeUpdateTime();
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ae14e5d0/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 c625fae..3dc2090 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
@@ -37,9 +37,11 @@ import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.security.AccessControlException;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.authorize.AccessControlList;
+import org.apache.hadoop.util.Time;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.Container;
+import org.apache.hadoop.yarn.api.records.ContainerExitStatus;
 import org.apache.hadoop.yarn.api.records.ContainerStatus;
 import org.apache.hadoop.yarn.api.records.Priority;
 import org.apache.hadoop.yarn.api.records.QueueACL;
@@ -63,7 +65,9 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceUsage;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedContainerChangeRequest;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplicationAttempt;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplicationAttempt.AMState;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerHealth;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerUtils;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.preemption.KillableContainer;
 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.server.resourcemanager.scheduler.policy.FifoOrderingPolicyForPendingApps;
@@ -823,6 +827,40 @@ public class LeafQueue extends AbstractCSQueue {
       assignment.setExcessReservation(null);
     }
   }
+
+  private void killToPreemptContainers(Resource clusterResource,
+      FiCaSchedulerNode node,
+      CSAssignment assignment) {
+    if (assignment.getContainersToKill() != null) {
+      StringBuilder sb = new StringBuilder("Killing containers: [");
+
+      for (RMContainer c : assignment.getContainersToKill()) {
+        FiCaSchedulerApp application = csContext.getApplicationAttempt(
+            c.getApplicationAttemptId());
+        LeafQueue q = application.getCSLeafQueue();
+        q.completedContainer(clusterResource, application, node, c, SchedulerUtils
+                .createPreemptedContainerStatus(c.getContainerId(),
+                    SchedulerUtils.PREEMPTED_CONTAINER), RMContainerEventType.KILL,
+            null, false);
+        sb.append("(container=" + c.getContainerId() + " resource=" + c
+            .getAllocatedResource() + ")");
+      }
+
+      sb.append("] for container=" + assignment.getAssignmentInformation()
+          .getFirstAllocatedOrReservedContainerId() + " resource=" + assignment
+          .getResource());
+      LOG.info(sb.toString());
+
+    }
+  }
+
+  private void setPreemptionAllowed(ResourceLimits limits, String nodePartition) {
+    // Set preemption-allowed:
+    // For leaf queue, only under-utilized queue is allowed to preempt resources from other queues
+    float usedCapacity = queueCapacities.getAbsoluteUsedCapacity(nodePartition);
+    float guaranteedCapacity = queueCapacities.getAbsoluteCapacity(nodePartition);
+    limits.setIsAllowPreemption(usedCapacity < guaranteedCapacity);
+  }
   
   @Override
   public synchronized CSAssignment assignContainers(Resource clusterResource,
@@ -835,6 +873,8 @@ public class LeafQueue extends AbstractCSQueue {
           + " #applications=" + orderingPolicy.getNumSchedulableEntities());
     }
 
+    setPreemptionAllowed(currentResourceLimits, node.getPartition());
+
     // Check for reserved resources
     RMContainer reservedContainer = node.getReservedContainer();
     if (reservedContainer != null) {
@@ -846,6 +886,7 @@ public class LeafQueue extends AbstractCSQueue {
                 currentResourceLimits, schedulingMode, reservedContainer);
         handleExcessReservedContainer(clusterResource, assignment, node,
             application);
+        killToPreemptContainers(clusterResource, node, assignment);
         return assignment;
       }
     }
@@ -907,6 +948,7 @@ public class LeafQueue extends AbstractCSQueue {
       
       handleExcessReservedContainer(clusterResource, assignment, node,
           application);
+      killToPreemptContainers(clusterResource, node, assignment);
 
       if (Resources.greaterThan(resourceCalculator, clusterResource, assigned,
           Resources.none())) {
@@ -1210,11 +1252,34 @@ public class LeafQueue extends AbstractCSQueue {
     }
   }
 
+  private void updateSchedulerHealthForCompletedContainer(
+      RMContainer rmContainer, ContainerStatus containerStatus) {
+    // Update SchedulerHealth for released / preempted container
+    SchedulerHealth schedulerHealth = csContext.getSchedulerHealth();
+    if (null == schedulerHealth) {
+      // Only do update if we have schedulerHealth
+      return;
+    }
+
+    if (containerStatus.getExitStatus() == ContainerExitStatus.PREEMPTED) {
+      schedulerHealth.updatePreemption(Time.now(), rmContainer.getAllocatedNode(),
+          rmContainer.getContainerId(), getQueuePath());
+      schedulerHealth.updateSchedulerPreemptionCounts(1);
+    } else {
+      schedulerHealth.updateRelease(csContext.getLastNodeUpdateTime(),
+          rmContainer.getAllocatedNode(), rmContainer.getContainerId(),
+          getQueuePath());
+    }
+  }
+
   @Override
   public void completedContainer(Resource clusterResource, 
       FiCaSchedulerApp application, FiCaSchedulerNode node, RMContainer rmContainer, 
       ContainerStatus containerStatus, RMContainerEventType event, CSQueue childQueue,
       boolean sortQueues) {
+    // Update SchedulerHealth for released / preempted container
+    updateSchedulerHealthForCompletedContainer(rmContainer, containerStatus);
+
     if (application != null) {
       // unreserve container increase request if it previously reserved.
       if (rmContainer.hasIncreaseReservation()) {
@@ -1265,6 +1330,10 @@ public class LeafQueue extends AbstractCSQueue {
           rmContainer, null, event, this, sortQueues);
       }
     }
+
+    // Notify PreemptionManager
+    csContext.getPreemptionManager().removeKillableContainer(
+        new KillableContainer(rmContainer, node.getPartition(), queueName));
   }
 
   synchronized void allocateResource(Resource clusterResource,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ae14e5d0/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 7cf5565..6fcd6c1 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
@@ -18,18 +18,6 @@
 
 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;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.TreeSet;
-
 import org.apache.commons.lang.StringUtils;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -49,6 +37,7 @@ import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.exceptions.InvalidResourceRequestException;
 import org.apache.hadoop.yarn.factories.RecordFactory;
 import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
+import org.apache.hadoop.yarn.nodelabels.RMNodeLabel;
 import org.apache.hadoop.yarn.security.AccessType;
 import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager;
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
@@ -57,12 +46,25 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerStat
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ActiveUsersManager;
 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.SchedContainerChangeRequest;
+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.fica.FiCaSchedulerApp;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerNode;
 import org.apache.hadoop.yarn.util.resource.Resources;
 
+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;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.TreeSet;
+
 @Private
 @Evolving
 public class ParentQueue extends AbstractCSQueue {
@@ -386,6 +388,11 @@ public class ParentQueue extends AbstractCSQueue {
     // if our queue cannot access this node, just return
     if (schedulingMode == SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY
         && !accessibleToPartition(node.getPartition())) {
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Skip this queue=" + getQueuePath()
+            + ", because it is not able to access partition=" + node
+            .getPartition());
+      }
       return CSAssignment.NULL_ASSIGNMENT;
     }
     
@@ -431,7 +438,7 @@ public class ParentQueue extends AbstractCSQueue {
               resourceCalculator, clusterResource, 
               assignedToChild.getResource(), Resources.none())) {
         // Track resource utilization for the parent-queue
-        super.allocateResource(clusterResource, assignedToChild.getResource(),
+        allocateResource(clusterResource, assignedToChild.getResource(),
             node.getPartition(), assignedToChild.isIncreasedAllocation());
         
         // Track resource utilization in this pass of the scheduler
@@ -494,29 +501,38 @@ public class ParentQueue extends AbstractCSQueue {
   }
 
   private boolean canAssign(Resource clusterResource, FiCaSchedulerNode node) {
-    return (node.getReservedContainer() == null) && 
-        Resources.greaterThanOrEqual(resourceCalculator, clusterResource, 
-            node.getUnallocatedResource(), minimumAllocation);
+    // Two conditions need to meet when trying to allocate:
+    // 1) Node doesn't have reserved container
+    // 2) Node's available-resource + killable-resource should > 0
+    return node.getReservedContainer() == null && Resources.greaterThanOrEqual(
+        resourceCalculator, clusterResource, Resources
+            .add(node.getUnallocatedResource(), node.getTotalKillableResources()),
+        minimumAllocation);
   }
-  
+
   private ResourceLimits getResourceLimitsOfChild(CSQueue child,
-      Resource clusterResource, ResourceLimits parentLimits) {
+      Resource clusterResource, ResourceLimits parentLimits,
+      String nodePartition) {
     // Set resource-limit of a given child, child.limit =
     // min(my.limit - my.used + child.used, child.max)
 
     // Parent available resource = parent-limit - parent-used-resource
-    Resource parentMaxAvailableResource =
-        Resources.subtract(parentLimits.getLimit(), getUsedResources());
+    Resource parentMaxAvailableResource = Resources.subtract(
+        parentLimits.getLimit(), queueUsage.getUsed(nodePartition));
+    // Deduct killable from used
+    Resources.addTo(parentMaxAvailableResource,
+        getTotalKillableResource(nodePartition));
 
     // Child's limit = parent-available-resource + child-used
-    Resource childLimit =
-        Resources.add(parentMaxAvailableResource, child.getUsedResources());
+    Resource childLimit = Resources.add(parentMaxAvailableResource,
+        child.getQueueResourceUsage().getUsed(nodePartition));
 
     // Get child's max resource
-    Resource childConfiguredMaxResource =
-        Resources.multiplyAndNormalizeDown(resourceCalculator, labelManager
-            .getResourceByLabel(RMNodeLabelsManager.NO_LABEL, clusterResource),
-            child.getAbsoluteMaximumCapacity(), minimumAllocation);
+    Resource childConfiguredMaxResource = Resources.multiplyAndNormalizeDown(
+        resourceCalculator,
+        labelManager.getResourceByLabel(nodePartition, clusterResource),
+        child.getQueueCapacities().getAbsoluteMaximumCapacity(nodePartition),
+        minimumAllocation);
 
     // Child's limit should be capped by child configured max resource
     childLimit =
@@ -568,7 +584,7 @@ public class ParentQueue extends AbstractCSQueue {
 
       // Get ResourceLimits of child queue before assign containers
       ResourceLimits childLimits =
-          getResourceLimitsOfChild(childQueue, cluster, limits);
+          getResourceLimitsOfChild(childQueue, cluster, limits, node.getPartition());
       
       assignment = childQueue.assignContainers(cluster, node, 
           childLimits, schedulingMode);
@@ -714,8 +730,8 @@ public class ParentQueue extends AbstractCSQueue {
     // Update all children
     for (CSQueue childQueue : childQueues) {
       // Get ResourceLimits of child queue before assign containers
-      ResourceLimits childLimits =
-          getResourceLimitsOfChild(childQueue, clusterResource, resourceLimits);     
+      ResourceLimits childLimits = getResourceLimitsOfChild(childQueue,
+          clusterResource, resourceLimits, RMNodeLabelsManager.NO_LABEL);
       childQueue.updateClusterResource(clusterResource, childLimits);
     }
     
@@ -738,8 +754,8 @@ public class ParentQueue extends AbstractCSQueue {
     synchronized (this) {
       FiCaSchedulerNode node =
           scheduler.getNode(rmContainer.getContainer().getNodeId());
-      super.allocateResource(clusterResource, rmContainer.getContainer()
-          .getResource(), node.getPartition(), false);
+      allocateResource(clusterResource,
+          rmContainer.getContainer().getResource(), node.getPartition(), false);
     }
     if (parent != null) {
       parent.recoverContainer(clusterResource, attempt, rmContainer);
@@ -766,7 +782,7 @@ public class ParentQueue extends AbstractCSQueue {
     if (application != null) {
       FiCaSchedulerNode node =
           scheduler.getNode(rmContainer.getContainer().getNodeId());
-      super.allocateResource(clusterResource, rmContainer.getContainer()
+      allocateResource(clusterResource, rmContainer.getContainer()
           .getResource(), node.getPartition(), false);
       LOG.info("movedContainer" + " queueMoveIn=" + getQueueName()
           + " usedCapacity=" + getUsedCapacity() + " absoluteUsedCapacity="
@@ -802,4 +818,79 @@ public class ParentQueue extends AbstractCSQueue {
   public synchronized int getNumApplications() {
     return numApplications;
   }
+
+  synchronized void allocateResource(Resource clusterResource,
+      Resource resource, String nodePartition, boolean changeContainerResource) {
+    super.allocateResource(clusterResource, resource, nodePartition,
+        changeContainerResource);
+
+    /**
+     * check if we need to kill (killable) containers if maximum resource violated.
+     * Doing this because we will deduct killable resource when going from root.
+     * For example:
+     * <pre>
+     *      Root
+     *      /   \
+     *     a     b
+     *   /  \
+     *  a1  a2
+     * </pre>
+     *
+     * a: max=10G, used=10G, killable=2G
+     * a1: used=8G, killable=2G
+     * a2: used=2G, pending=2G, killable=0G
+     *
+     * When we get queue-a to allocate resource, even if queue-a
+     * reaches its max resource, we deduct its used by killable, so we can allocate
+     * at most 2G resources. ResourceLimits passed down to a2 has headroom set to 2G.
+     *
+     * If scheduler finds a 2G available resource in existing cluster, and assigns it
+     * to a2, now a2's used= 2G + 2G = 4G, and a's used = 8G + 4G = 12G > 10G
+     *
+     * When this happens, we have to preempt killable container (on same or different
+     * nodes) of parent queue to avoid violating parent's max resource.
+     */
+    if (getQueueCapacities().getAbsoluteMaximumCapacity(nodePartition)
+        < getQueueCapacities().getAbsoluteUsedCapacity(nodePartition)) {
+      killContainersToEnforceMaxQueueCapacity(nodePartition, clusterResource);
+    }
+  }
+
+  private void killContainersToEnforceMaxQueueCapacity(String partition,
+      Resource clusterResource) {
+    Iterator<RMContainer> killableContainerIter = getKillableContainers(
+        partition);
+    if (!killableContainerIter.hasNext()) {
+      return;
+    }
+
+    Resource partitionResource = labelManager.getResourceByLabel(partition,
+        null);
+    Resource maxResource = Resources.multiply(partitionResource,
+        getQueueCapacities().getAbsoluteMaximumCapacity(partition));
+
+    while (Resources.greaterThan(resourceCalculator, partitionResource,
+        queueUsage.getUsed(partition), maxResource)) {
+      RMContainer toKillContainer = killableContainerIter.next();
+      FiCaSchedulerApp attempt = csContext.getApplicationAttempt(
+          toKillContainer.getContainerId().getApplicationAttemptId());
+      FiCaSchedulerNode node = csContext.getNode(
+          toKillContainer.getAllocatedNode());
+      if (null != attempt && null != node) {
+        LeafQueue lq = attempt.getCSLeafQueue();
+        lq.completedContainer(clusterResource, attempt, node, toKillContainer,
+            SchedulerUtils.createPreemptedContainerStatus(
+                toKillContainer.getContainerId(),
+                SchedulerUtils.PREEMPTED_CONTAINER), RMContainerEventType.KILL,
+            null, false);
+        LOG.info("Killed container=" + toKillContainer.getContainerId()
+            + " from queue=" + lq.getQueueName() + " to make queue=" + this
+            .getQueueName() + "'s max-capacity enforced");
+      }
+
+      if (!killableContainerIter.hasNext()) {
+        break;
+      }
+    }
+  }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ae14e5d0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/allocator/AbstractContainerAllocator.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/allocator/AbstractContainerAllocator.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/allocator/AbstractContainerAllocator.java
index ee01bd1..afac235 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/allocator/AbstractContainerAllocator.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/allocator/AbstractContainerAllocator.java
@@ -108,6 +108,8 @@ public abstract class AbstractContainerAllocator {
           assignment.setFulfilledReservation(true);
         }
       }
+
+      assignment.setContainersToKill(result.getToKillContainers());
     }
     
     return assignment;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ae14e5d0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/allocator/ContainerAllocation.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/allocator/ContainerAllocation.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/allocator/ContainerAllocation.java
index 1df9410..8f749f6 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/allocator/ContainerAllocation.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/allocator/ContainerAllocation.java
@@ -19,11 +19,14 @@
 package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.allocator;
 
 import org.apache.hadoop.yarn.api.records.Container;
+import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.NodeType;
 import org.apache.hadoop.yarn.util.resource.Resources;
 
+import java.util.List;
+
 public class ContainerAllocation {
   /**
    * Skip the locality (e.g. node-local, rack-local, any), and look at other
@@ -56,6 +59,7 @@ public class ContainerAllocation {
   NodeType containerNodeType = NodeType.NODE_LOCAL;
   NodeType requestNodeType = NodeType.NODE_LOCAL;
   Container updatedContainer;
+  private List<RMContainer> toKillContainers;
 
   public ContainerAllocation(RMContainer containerToBeUnreserved,
       Resource resourceToBeAllocated, AllocationState state) {
@@ -86,4 +90,12 @@ public class ContainerAllocation {
   public Container getUpdatedContainer() {
     return updatedContainer;
   }
+
+  public void setToKillContainers(List<RMContainer> toKillContainers) {
+    this.toKillContainers = toKillContainers;
+  }
+
+  public List<RMContainer> getToKillContainers() {
+    return toKillContainers;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ae14e5d0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/allocator/RegularContainerAllocator.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/allocator/RegularContainerAllocator.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/allocator/RegularContainerAllocator.java
index e168edf..a5ca2d8 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/allocator/RegularContainerAllocator.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/allocator/RegularContainerAllocator.java
@@ -42,6 +42,9 @@ import org.apache.hadoop.yarn.server.utils.BuilderUtils;
 import org.apache.hadoop.yarn.util.resource.ResourceCalculator;
 import org.apache.hadoop.yarn.util.resource.Resources;
 
+import java.util.ArrayList;
+import java.util.List;
+
 /**
  * Allocate normal (new) containers, considers locality/label, etc. Using
  * delayed scheduling mechanism to get better locality allocation.
@@ -435,9 +438,6 @@ public class RegularContainerAllocator extends AbstractContainerAllocator {
       return ContainerAllocation.LOCALITY_SKIPPED;
     }
 
-    assert Resources.greaterThan(
-        rc, clusterResource, available, Resources.none());
-
     boolean shouldAllocOrReserveNewContainer = shouldAllocOrReserveNewContainer(
         priority, capability);
 
@@ -460,6 +460,29 @@ public class RegularContainerAllocator extends AbstractContainerAllocator {
     boolean reservationsContinueLooking =
         application.getCSLeafQueue().getReservationContinueLooking();
 
+    // Check if we need to kill some containers to allocate this one
+    List<RMContainer> toKillContainers = null;
+    if (availableContainers == 0 && currentResoureLimits.isAllowPreemption()) {
+      Resource availableAndKillable = Resources.clone(available);
+      for (RMContainer killableContainer : node
+          .getKillableContainers().values()) {
+        if (null == toKillContainers) {
+          toKillContainers = new ArrayList<>();
+        }
+        toKillContainers.add(killableContainer);
+        Resources.addTo(availableAndKillable,
+                        killableContainer.getAllocatedResource());
+        if (Resources.fitsIn(rc,
+                             clusterResource,
+                             capability,
+                             availableAndKillable)) {
+          // Stop if we find enough spaces
+          availableContainers = 1;
+          break;
+        }
+      }
+    }
+
     if (availableContainers > 0) {
       // Allocate...
       // We will only do continuous reservation when this is not allocated from
@@ -499,12 +522,12 @@ public class RegularContainerAllocator extends AbstractContainerAllocator {
           new ContainerAllocation(unreservedContainer, request.getCapability(),
               AllocationState.ALLOCATED);
       result.containerNodeType = type;
+      result.setToKillContainers(toKillContainers);
       return result;
     } else {
       // if we are allowed to allocate but this node doesn't have space, reserve
       // it or if this was an already a reserved container, reserve it again
       if (shouldAllocOrReserveNewContainer || rmContainer != null) {
-
         if (reservationsContinueLooking && rmContainer == null) {
           // we could possibly ignoring queue capacity or user limits when
           // reservationsContinueLooking is set. Make sure we didn't need to
@@ -522,6 +545,7 @@ public class RegularContainerAllocator extends AbstractContainerAllocator {
             new ContainerAllocation(null, request.getCapability(),
                 AllocationState.RESERVED);
         result.containerNodeType = type;
+        result.setToKillContainers(null);
         return result;
       }
       // Skip the locality request
@@ -613,8 +637,7 @@ public class RegularContainerAllocator extends AbstractContainerAllocator {
   }
 
   ContainerAllocation doAllocation(ContainerAllocation allocationResult,
-      Resource clusterResource, FiCaSchedulerNode node,
-      SchedulingMode schedulingMode, Priority priority,
+      FiCaSchedulerNode node, Priority priority,
       RMContainer reservedContainer) {
     // Create the container if necessary
     Container container =
@@ -678,9 +701,7 @@ public class RegularContainerAllocator extends AbstractContainerAllocator {
 
     if (AllocationState.ALLOCATED == result.state
         || AllocationState.RESERVED == result.state) {
-      result =
-          doAllocation(result, clusterResource, node, schedulingMode, priority,
-              reservedContainer);
+      result = doAllocation(result, node, priority, reservedContainer);
     }
 
     return result;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ae14e5d0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/preemption/KillableContainer.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/preemption/KillableContainer.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/preemption/KillableContainer.java
new file mode 100644
index 0000000..675b0b4
--- /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/preemption/KillableContainer.java
@@ -0,0 +1,45 @@
+/**
+ * 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
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
+ * 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.preemption;
+
+import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
+
+public class KillableContainer {
+  RMContainer container;
+  String partition;
+  String leafQueueName;
+
+  public KillableContainer(RMContainer container, String partition, String leafQueueName) {
+    this.container = container;
+    this.partition = partition;
+    this.leafQueueName = leafQueueName;
+  }
+
+  public RMContainer getRMContainer() {
+    return this.container;
+  }
+
+  public String getNodePartition() {
+    return this.partition;
+  }
+
+  public String getLeafQueueName() {
+    return this.leafQueueName;
+  }
+}


[21/46] hadoop git commit: YARN-4817. Change Log Level to DEBUG for putDomain call in ATS 1.5. (Xuan Gong via gtcarrera9)

Posted by ae...@apache.org.
YARN-4817. Change Log Level to DEBUG for putDomain call in ATS 1.5. (Xuan Gong via gtcarrera9)


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

Branch: refs/heads/HDFS-7240
Commit: 5176a69868484f864c977a6cd13923167082407c
Parents: 3a0b695
Author: Li Lu <gt...@apache.org>
Authored: Tue Mar 15 11:47:39 2016 -0700
Committer: Li Lu <gt...@apache.org>
Committed: Tue Mar 15 11:47:39 2016 -0700

----------------------------------------------------------------------
 .../hadoop/yarn/client/api/impl/FileSystemTimelineWriter.java  | 6 ++++--
 1 file changed, 4 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/5176a698/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/impl/FileSystemTimelineWriter.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/impl/FileSystemTimelineWriter.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/impl/FileSystemTimelineWriter.java
index 44cad60..c494f65 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/impl/FileSystemTimelineWriter.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/impl/FileSystemTimelineWriter.java
@@ -306,8 +306,10 @@ public class FileSystemTimelineWriter extends TimelineWriter{
     Path domainLogPath =
         new Path(createAttemptDir(appAttemptId), DOMAIN_LOG_PREFIX
             + appAttemptId.toString());
-    LOG.info("Writing domains for " + appAttemptId.toString() + " to "
-        + domainLogPath);
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Writing domains for " + appAttemptId.toString() + " to "
+          + domainLogPath);
+    }
     this.logFDsCache.writeDomainLog(
         fs, domainLogPath, objMapper, domain, isAppendSupported);
   }


[14/46] hadoop git commit: HADOOP-12923. Move the test code in ipc.Client to test.

Posted by ae...@apache.org.
HADOOP-12923. Move the test code in ipc.Client to test.


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

Branch: refs/heads/HDFS-7240
Commit: 1898810cda83e6d273a2963b56ed499c0fb91118
Parents: 9a43094
Author: Tsz-Wo Nicholas Sze <sz...@hortonworks.com>
Authored: Mon Mar 14 15:48:01 2016 -0700
Committer: Tsz-Wo Nicholas Sze <sz...@hortonworks.com>
Committed: Mon Mar 14 15:48:01 2016 -0700

----------------------------------------------------------------------
 .../main/java/org/apache/hadoop/ipc/Client.java | 109 +------------------
 .../apache/hadoop/ipc/WritableRpcEngine.java    |   2 +-
 .../java/org/apache/hadoop/ipc/TestIPC.java     | 101 +++++++++--------
 .../hadoop/ipc/TestIPCServerResponder.java      |  30 +++--
 .../java/org/apache/hadoop/ipc/TestSaslRPC.java |  18 +--
 5 files changed, 90 insertions(+), 170 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/1898810c/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Client.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Client.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Client.java
index 8d87957..7e6c7e3 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Client.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Client.java
@@ -1120,12 +1120,11 @@ public class Client {
         if (LOG.isDebugEnabled())
           LOG.debug(getName() + " got value #" + callId);
 
-        Call call = calls.get(callId);
         RpcStatusProto status = header.getStatus();
         if (status == RpcStatusProto.SUCCESS) {
           Writable value = ReflectionUtils.newInstance(valueClass, conf);
           value.readFields(in);                 // read value
-          calls.remove(callId);
+          final Call call = calls.remove(callId);
           call.setRpcResponse(value);
           
           // verify that length was correct
@@ -1157,7 +1156,7 @@ public class Client {
           }
           RemoteException re = new RemoteException(exceptionClassName, errorMsg, erCode);
           if (status == RpcStatusProto.ERROR) {
-            calls.remove(callId);
+            final Call call = calls.remove(callId);
             call.setException(re);
           } else if (status == RpcStatusProto.FATAL) {
             // Close the connection
@@ -1288,85 +1287,6 @@ public class Client {
     clientExcecutorFactory.unrefAndCleanup();
   }
 
-  /**
-   * Same as {@link #call(RPC.RpcKind, Writable, ConnectionId)}
-   *  for RPC_BUILTIN
-   */
-  public Writable call(Writable param, InetSocketAddress address)
-      throws IOException {
-    ConnectionId remoteId = ConnectionId.getConnectionId(address, null, null, 0,
-        conf);
-    return call(RpcKind.RPC_BUILTIN, param, remoteId);
-  }
-
-  /**
-   * Same as {@link #call(RPC.RpcKind, Writable, InetSocketAddress,
-   * Class, UserGroupInformation, int, Configuration)}
-   * except that rpcKind is writable.
-   */
-  public Writable call(Writable param, InetSocketAddress addr,
-      Class<?> protocol, UserGroupInformation ticket,
-      int rpcTimeout, Configuration conf) throws IOException {
-    ConnectionId remoteId = ConnectionId.getConnectionId(addr, protocol,
-        ticket, rpcTimeout, conf);
-    return call(RPC.RpcKind.RPC_BUILTIN, param, remoteId);
-  }
-  
-  /**
-   * Same as {@link #call(Writable, InetSocketAddress,
-   * Class, UserGroupInformation, int, Configuration)}
-   * except that specifying serviceClass.
-   */
-  public Writable call(Writable param, InetSocketAddress addr,
-      Class<?> protocol, UserGroupInformation ticket,
-      int rpcTimeout, int serviceClass, Configuration conf)
-      throws IOException {
-    ConnectionId remoteId = ConnectionId.getConnectionId(addr, protocol,
-        ticket, rpcTimeout, conf);
-    return call(RPC.RpcKind.RPC_BUILTIN, param, remoteId, serviceClass);
-  }
-
-  /**
-   * Make a call, passing <code>param</code>, to the IPC server running at
-   * <code>address</code> which is servicing the <code>protocol</code> protocol,
-   * with the <code>ticket</code> credentials, <code>rpcTimeout</code> as
-   * timeout and <code>conf</code> as conf for this connection, returning the
-   * value. Throws exceptions if there are network problems or if the remote
-   * code threw an exception.
-   */
-  public Writable call(RPC.RpcKind rpcKind, Writable param, InetSocketAddress addr, 
-                       Class<?> protocol, UserGroupInformation ticket,
-                       int rpcTimeout, Configuration conf) throws IOException {
-    ConnectionId remoteId = ConnectionId.getConnectionId(addr, protocol,
-        ticket, rpcTimeout, conf);
-    return call(rpcKind, param, remoteId);
-  }
-  
-  /**
-   * Same as {link {@link #call(RPC.RpcKind, Writable, ConnectionId)}
-   * except the rpcKind is RPC_BUILTIN
-   */
-  public Writable call(Writable param, ConnectionId remoteId)
-      throws IOException {
-     return call(RPC.RpcKind.RPC_BUILTIN, param, remoteId);
-  }
-  
-  /**
-   * Make a call, passing <code>rpcRequest</code>, to the IPC server defined by
-   * <code>remoteId</code>, returning the rpc respond.
-   *
-   * @param rpcKind
-   * @param rpcRequest -  contains serialized method and method parameters
-   * @param remoteId - the target rpc server
-   * @returns the rpc response
-   * Throws exceptions if there are network problems or if the remote code 
-   * threw an exception.
-   */
-  public Writable call(RPC.RpcKind rpcKind, Writable rpcRequest,
-      ConnectionId remoteId) throws IOException {
-    return call(rpcKind, rpcRequest, remoteId, RPC.RPC_SERVICE_CLASS_DEFAULT);
-  }
-
   /** 
    * Make a call, passing <code>rpcRequest</code>, to the IPC server defined by
    * <code>remoteId</code>, returning the rpc respond.
@@ -1390,23 +1310,6 @@ public class Client {
   /**
    * Make a call, passing <code>rpcRequest</code>, to the IPC server defined by
    * <code>remoteId</code>, returning the rpc response.
-   * 
-   * @param rpcKind
-   * @param rpcRequest -  contains serialized method and method parameters
-   * @param remoteId - the target rpc server
-   * @param serviceClass - service class for RPC
-   * @returns the rpc response
-   * Throws exceptions if there are network problems or if the remote code 
-   * threw an exception.
-   */
-  public Writable call(RPC.RpcKind rpcKind, Writable rpcRequest,
-      ConnectionId remoteId, int serviceClass) throws IOException {
-    return call(rpcKind, rpcRequest, remoteId, serviceClass, null);
-  }
-
-  /**
-   * Make a call, passing <code>rpcRequest</code>, to the IPC server defined by
-   * <code>remoteId</code>, returning the rpc response.
    *
    * @param rpcKind
    * @param rpcRequest -  contains serialized method and method parameters
@@ -1418,7 +1321,7 @@ public class Client {
    * Throws exceptions if there are network problems or if the remote code
    * threw an exception.
    */
-  public Writable call(RPC.RpcKind rpcKind, Writable rpcRequest,
+  Writable call(RPC.RpcKind rpcKind, Writable rpcRequest,
       ConnectionId remoteId, int serviceClass,
       AtomicBoolean fallbackToSimpleAuth) throws IOException {
     final Call call = createCall(rpcKind, rpcRequest);
@@ -1620,12 +1523,6 @@ public class Client {
       return saslQop;
     }
     
-    static ConnectionId getConnectionId(InetSocketAddress addr,
-        Class<?> protocol, UserGroupInformation ticket, int rpcTimeout,
-        Configuration conf) throws IOException {
-      return getConnectionId(addr, protocol, ticket, rpcTimeout, null, conf);
-    }
-
     /**
      * Returns a ConnectionId object. 
      * @param addr Remote address for the connection.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1898810c/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/WritableRpcEngine.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/WritableRpcEngine.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/WritableRpcEngine.java
index 46f33ba..a1db6be 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/WritableRpcEngine.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/WritableRpcEngine.java
@@ -221,7 +221,7 @@ public class WritableRpcEngine implements RpcEngine {
                    int rpcTimeout, AtomicBoolean fallbackToSimpleAuth)
         throws IOException {
       this.remoteId = Client.ConnectionId.getConnectionId(address, protocol,
-          ticket, rpcTimeout, conf);
+          ticket, rpcTimeout, null, conf);
       this.client = CLIENTS.getClient(conf, factory);
       this.fallbackToSimpleAuth = fallbackToSimpleAuth;
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1898810c/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestIPC.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestIPC.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestIPC.java
index 78dcdcd..d658182 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestIPC.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestIPC.java
@@ -55,7 +55,6 @@ import java.util.concurrent.atomic.AtomicInteger;
 
 import javax.net.SocketFactory;
 
-import com.google.common.base.Supplier;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.commons.logging.impl.Log4JLogger;
@@ -63,7 +62,6 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
 import org.apache.hadoop.io.IOUtils;
-import org.apache.hadoop.io.IntWritable;
 import org.apache.hadoop.io.LongWritable;
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.io.retry.DefaultFailoverProxyProvider;
@@ -78,9 +76,9 @@ import org.apache.hadoop.ipc.Server.Connection;
 import org.apache.hadoop.ipc.protobuf.RpcHeaderProtos.RpcResponseHeaderProto;
 import org.apache.hadoop.net.ConnectTimeoutException;
 import org.apache.hadoop.net.NetUtils;
+import org.apache.hadoop.security.token.SecretManager.InvalidToken;
 import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.util.StringUtils;
-import org.apache.hadoop.security.token.SecretManager.InvalidToken;
 import org.apache.log4j.Level;
 import org.junit.Assert;
 import org.junit.Assume;
@@ -91,6 +89,7 @@ import org.mockito.internal.util.reflection.Whitebox;
 import org.mockito.invocation.InvocationOnMock;
 import org.mockito.stubbing.Answer;
 
+import com.google.common.base.Supplier;
 import com.google.common.primitives.Bytes;
 import com.google.common.primitives.Ints;
 
@@ -122,6 +121,33 @@ public class TestIPC {
   /** Directory where we can count open file descriptors on Linux */
   private static final File FD_DIR = new File("/proc/self/fd");
 
+  static ConnectionId getConnectionId(InetSocketAddress addr, int rpcTimeout,
+      Configuration conf) throws IOException {
+    return ConnectionId.getConnectionId(addr, null, null, rpcTimeout, null,
+        conf);
+  }
+
+  static Writable call(Client client, InetSocketAddress addr,
+      int serviceClass, Configuration conf) throws IOException {
+    final LongWritable param = new LongWritable(RANDOM.nextLong());
+    final ConnectionId remoteId = getConnectionId(addr, MIN_SLEEP_TIME, conf);
+    return client.call(RPC.RpcKind.RPC_BUILTIN, param, remoteId, serviceClass,
+        null);
+  }
+
+  static LongWritable call(Client client, long param, InetSocketAddress addr,
+      Configuration conf) throws IOException {
+    return call(client, new LongWritable(param), addr, 0, conf);
+  }
+
+  static LongWritable call(Client client, LongWritable param,
+      InetSocketAddress addr, int rpcTimeout, Configuration conf)
+          throws IOException {
+    final ConnectionId remoteId = getConnectionId(addr, rpcTimeout, conf);
+    return (LongWritable)client.call(RPC.RpcKind.RPC_BUILTIN, param, remoteId,
+        RPC.RPC_SERVICE_CLASS_DEFAULT, null);
+  }
+
   private static class TestServer extends Server {
     // Tests can set callListener to run a piece of code each time the server
     // receives a call.  This code executes on the server thread, so it has
@@ -183,10 +209,9 @@ public class TestIPC {
     public void run() {
       for (int i = 0; i < count; i++) {
         try {
-          LongWritable param = new LongWritable(RANDOM.nextLong());
-          LongWritable value =
-            (LongWritable)client.call(param, server, null, null, 0, conf);
-          if (!param.equals(value)) {
+          final long param = RANDOM.nextLong();
+          LongWritable value = call(client, param, server, conf);
+          if (value.get() != param) {
             LOG.fatal("Call failed!");
             failed = true;
             break;
@@ -226,9 +251,8 @@ public class TestIPC {
     @Override
     public Object invoke(Object proxy, Method method, Object[] args)
         throws Throwable {
-      LongWritable param = new LongWritable(RANDOM.nextLong());
-      LongWritable value = (LongWritable) client.call(param,
-          NetUtils.getConnectAddress(server), null, null, 0, conf);
+      LongWritable value = call(client, RANDOM.nextLong(),
+          NetUtils.getConnectAddress(server), conf);
       return returnValue(value);
     }
 
@@ -298,8 +322,7 @@ public class TestIPC {
     Client client = new Client(LongWritable.class, conf);
     InetSocketAddress address = new InetSocketAddress("127.0.0.1", 10);
     try {
-      client.call(new LongWritable(RANDOM.nextLong()),
-              address, null, null, 0, conf);
+      call(client, RANDOM.nextLong(), address, conf);
       fail("Expected an exception to have been thrown");
     } catch (IOException e) {
       String message = e.getMessage();
@@ -412,7 +435,7 @@ public class TestIPC {
       LongWritable param = clientParamClass.newInstance();
 
       try {
-        client.call(param, addr, null, null, 0, conf);
+        call(client, param, addr, 0, conf);
         fail("Expected an exception to have been thrown");
       } catch (Throwable t) {
         assertExceptionContains(t, "Injected fault");
@@ -422,7 +445,7 @@ public class TestIPC {
       // ie the internal state of the client or server should not be broken
       // by the failed call
       WRITABLE_FAULTS_ENABLED = false;
-      client.call(param, addr, null, null, 0, conf);
+      call(client, param, addr, 0, conf);
       
     } finally {
       client.stop();
@@ -536,8 +559,7 @@ public class TestIPC {
     
     InetSocketAddress address = new InetSocketAddress("127.0.0.1", 10);
     try {
-      client.call(new LongWritable(RANDOM.nextLong()),
-              address, null, null, 0, conf);
+      call(client, RANDOM.nextLong(), address, conf);
       fail("Expected an exception to have been thrown");
     } catch (IOException e) {
       assertTrue(e.getMessage().contains("Injected fault"));
@@ -574,8 +596,7 @@ public class TestIPC {
       // Call should fail due to injected exception.
       InetSocketAddress address = NetUtils.getConnectAddress(server);
       try {
-        client.call(new LongWritable(RANDOM.nextLong()),
-                address, null, null, 0, conf);
+        call(client, RANDOM.nextLong(), address, conf);
         fail("Expected an exception to have been thrown");
       } catch (Exception e) {
         LOG.info("caught expected exception", e);
@@ -586,8 +607,7 @@ public class TestIPC {
       // (i.e. it should not have cached a half-constructed connection)
   
       Mockito.reset(spyFactory);
-      client.call(new LongWritable(RANDOM.nextLong()),
-          address, null, null, 0, conf);
+      call(client, RANDOM.nextLong(), address, conf);
     } finally {
       client.stop();
       server.stop();
@@ -605,15 +625,15 @@ public class TestIPC {
     Client client = new Client(LongWritable.class, conf);
     // set timeout to be less than MIN_SLEEP_TIME
     try {
-      client.call(new LongWritable(RANDOM.nextLong()),
-              addr, null, null, MIN_SLEEP_TIME/2, conf);
+      call(client, new LongWritable(RANDOM.nextLong()), addr,
+          MIN_SLEEP_TIME / 2, conf);
       fail("Expected an exception to have been thrown");
     } catch (SocketTimeoutException e) {
       LOG.info("Get a SocketTimeoutException ", e);
     }
     // set timeout to be bigger than 3*ping interval
-    client.call(new LongWritable(RANDOM.nextLong()),
-        addr, null, null, 3*PING_INTERVAL+MIN_SLEEP_TIME, conf);
+    call(client, new LongWritable(RANDOM.nextLong()), addr,
+        3 * PING_INTERVAL + MIN_SLEEP_TIME, conf);
     client.stop();
   }
 
@@ -629,8 +649,8 @@ public class TestIPC {
     Client client = new Client(LongWritable.class, conf);
     // set the rpc timeout to twice the MIN_SLEEP_TIME
     try {
-      client.call(new LongWritable(RANDOM.nextLong()),
-              addr, null, null, MIN_SLEEP_TIME*2, conf);
+      call(client, new LongWritable(RANDOM.nextLong()), addr,
+          MIN_SLEEP_TIME * 2, conf);
       fail("Expected an exception to have been thrown");
     } catch (SocketTimeoutException e) {
       LOG.info("Get a SocketTimeoutException ", e);
@@ -743,8 +763,8 @@ public class TestIPC {
         public void run() {
           Client client = new Client(LongWritable.class, conf);
           try {
-            client.call(new LongWritable(Thread.currentThread().getId()),
-                addr, null, null, 60000, conf);
+            call(client, new LongWritable(Thread.currentThread().getId()),
+                addr, 60000, conf);
           } catch (Throwable e) {
             LOG.error(e);
             failures.incrementAndGet();
@@ -875,8 +895,7 @@ public class TestIPC {
           public void run() {
             Client client = new Client(LongWritable.class, clientConf);
             try {
-              client.call(new LongWritable(Thread.currentThread().getId()),
-                  addr, null, null, 0, clientConf);
+              call(client, Thread.currentThread().getId(), addr, clientConf);
               callReturned.countDown();
               Thread.sleep(10000);
             } catch (IOException e) {
@@ -931,16 +950,15 @@ public class TestIPC {
       }
     }
   }
-  
+
   /**
    * Make a call from a client and verify if header info is changed in server side
    */
-  private void callAndVerify(Server server, InetSocketAddress addr,
+  private static void callAndVerify(Server server, InetSocketAddress addr,
       int serviceClass, boolean noChanged) throws IOException{
     Client client = new Client(LongWritable.class, conf);
 
-    client.call(new LongWritable(RANDOM.nextLong()),
-        addr, null, null, MIN_SLEEP_TIME, serviceClass, conf);
+    call(client, addr, serviceClass, conf);
     Connection connection = server.getConnections()[0];
     int serviceClass2 = connection.getServiceClass();
     assertFalse(noChanged ^ serviceClass == serviceClass2);
@@ -956,13 +974,11 @@ public class TestIPC {
 
     // start client
     Client client = new Client(LongWritable.class, conf);
-    client.call(new LongWritable(RANDOM.nextLong()),
-        addr, null, null, MIN_SLEEP_TIME, 0, conf);
+    call(client, addr, 0, conf);
     client.stop();
  
     // This call should throw IOException.
-    client.call(new LongWritable(RANDOM.nextLong()),
-        addr, null, null, MIN_SLEEP_TIME, 0, conf);
+    call(client, addr, 0, conf);
   }
 
   /**
@@ -992,7 +1008,7 @@ public class TestIPC {
   @Test(timeout=30000)
   public void testInterrupted() {
     Client client = new Client(LongWritable.class, conf);
-    client.getClientExecutor().submit(new Runnable() {
+    Client.getClientExecutor().submit(new Runnable() {
       public void run() {
         while(true);
       }
@@ -1007,7 +1023,7 @@ public class TestIPC {
       Assert.fail("The Client did not interrupt after handling an Interrupted Exception");
     }
     // Clear Thread interrupt
-    Thread.currentThread().interrupted();
+    Thread.interrupted();
   }
 
   private long countOpenFileDescriptors() {
@@ -1363,11 +1379,10 @@ public class TestIPC {
       int maxTimeoutRetries) throws IOException {
     SocketFactory mockFactory = Mockito.mock(SocketFactory.class);
     doThrow(new ConnectTimeoutException("fake")).when(mockFactory).createSocket();
-    Client client = new Client(IntWritable.class, conf, mockFactory);
+    Client client = new Client(LongWritable.class, conf, mockFactory);
     InetSocketAddress address = new InetSocketAddress("127.0.0.1", 9090);
     try {
-      client.call(new IntWritable(RANDOM.nextInt()), address, null, null, 0,
-          conf);
+      call(client, RANDOM.nextLong(), address, conf);
       fail("Not throwing the SocketTimeoutException");
     } catch (SocketTimeoutException e) {
       Mockito.verify(mockFactory, Mockito.times(maxTimeoutRetries))

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1898810c/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestIPCServerResponder.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestIPCServerResponder.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestIPCServerResponder.java
index a3bf995..546cb8f 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestIPCServerResponder.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestIPCServerResponder.java
@@ -39,6 +39,8 @@ import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.io.BytesWritable;
 import org.apache.hadoop.io.IntWritable;
 import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.ipc.Client.ConnectionId;
+import org.apache.hadoop.ipc.RPC.RpcKind;
 import org.apache.hadoop.ipc.Server.Call;
 import org.apache.hadoop.net.NetUtils;
 import org.junit.Assert;
@@ -66,6 +68,14 @@ public class TestIPCServerResponder {
       BYTES[i] = (byte) ('a' + (i % 26));
   }
 
+  static Writable call(Client client, Writable param,
+      InetSocketAddress address) throws IOException {
+    final ConnectionId remoteId = ConnectionId.getConnectionId(address, null,
+        null, 0, null, conf);
+    return client.call(RpcKind.RPC_BUILTIN, param, remoteId,
+        RPC.RPC_SERVICE_CLASS_DEFAULT, null);
+  }
+
   private static class TestServer extends Server {
 
     private boolean sleep;
@@ -113,7 +123,7 @@ public class TestIPCServerResponder {
           byte[] bytes = new byte[byteSize];
           System.arraycopy(BYTES, 0, bytes, 0, byteSize);
           Writable param = new BytesWritable(bytes);
-          client.call(param, address);
+          call(client, param, address);
           Thread.sleep(RANDOM.nextInt(20));
         } catch (Exception e) {
           LOG.fatal("Caught Exception", e);
@@ -209,17 +219,16 @@ public class TestIPCServerResponder {
 
     // calls should return immediately, check the sequence number is
     // increasing
-    assertEquals(0,
-        ((IntWritable)client.call(wait0, address)).get());
-    assertEquals(1,
-        ((IntWritable)client.call(wait0, address)).get());
+    assertEquals(0, ((IntWritable)call(client, wait0, address)).get());
+    assertEquals(1, ((IntWritable)call(client, wait0, address)).get());
 
     // do a call in the background that will have a deferred response
     final ExecutorService exec = Executors.newCachedThreadPool();
     Future<Integer> future1 = exec.submit(new Callable<Integer>() {
       @Override
       public Integer call() throws IOException {
-        return ((IntWritable)client.call(wait1, address)).get();
+        return ((IntWritable)TestIPCServerResponder.call(
+            client, wait1, address)).get();
       }
     });
     // make sure it blocked
@@ -237,14 +246,14 @@ public class TestIPCServerResponder {
 
     // proves the handler isn't tied up, and that the prior sequence number
     // was consumed
-    assertEquals(3,
-        ((IntWritable)client.call(wait0, address)).get());
+    assertEquals(3, ((IntWritable)call(client, wait0, address)).get());
 
     // another call with wait count of 2
     Future<Integer> future2 = exec.submit(new Callable<Integer>() {
       @Override
       public Integer call() throws IOException {
-        return ((IntWritable)client.call(wait2, address)).get();
+        return ((IntWritable)TestIPCServerResponder.call(
+            client, wait2, address)).get();
       }
     });
     // make sure it blocked
@@ -286,8 +295,7 @@ public class TestIPCServerResponder {
     assertFalse(future2.isDone());
 
     // call should return immediately
-    assertEquals(5,
-        ((IntWritable)client.call(wait0, address)).get());
+    assertEquals(5, ((IntWritable)call(client, wait0, address)).get());
 
     // trigger last waiting call
     waitingCalls[1].sendResponse();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1898810c/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestSaslRPC.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestSaslRPC.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestSaslRPC.java
index a32ea2c..8df3b1d 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestSaslRPC.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestSaslRPC.java
@@ -18,8 +18,8 @@
 
 package org.apache.hadoop.ipc;
 
-import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHENTICATION;
 import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.HADOOP_RPC_PROTECTION;
+import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHENTICATION;
 import static org.apache.hadoop.security.SaslRpcServer.AuthMethod.KERBEROS;
 import static org.apache.hadoop.security.SaslRpcServer.AuthMethod.SIMPLE;
 import static org.apache.hadoop.security.SaslRpcServer.AuthMethod.TOKEN;
@@ -42,16 +42,13 @@ import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.HashMap;
-import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
 import java.util.concurrent.Callable;
-import java.util.concurrent.ExecutionException;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;
 import java.util.concurrent.Future;
-import java.util.concurrent.ThreadLocalRandom;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.TimeoutException;
 import java.util.concurrent.atomic.AtomicInteger;
@@ -454,7 +451,12 @@ public class TestSaslRPC {
       }
     }
   }
-  
+
+  static ConnectionId getConnectionId(Configuration conf) throws IOException {
+    return ConnectionId.getConnectionId(new InetSocketAddress(0),
+        TestSaslProtocol.class, null, 0, null, conf);
+  }
+
   @Test
   public void testPingInterval() throws Exception {
     Configuration newConf = new Configuration(conf);
@@ -464,14 +466,12 @@ public class TestSaslRPC {
 
     // set doPing to true
     newConf.setBoolean(CommonConfigurationKeys.IPC_CLIENT_PING_KEY, true);
-    ConnectionId remoteId = ConnectionId.getConnectionId(
-        new InetSocketAddress(0), TestSaslProtocol.class, null, 0, newConf);
+    ConnectionId remoteId = getConnectionId(newConf);
     assertEquals(CommonConfigurationKeys.IPC_PING_INTERVAL_DEFAULT,
         remoteId.getPingInterval());
     // set doPing to false
     newConf.setBoolean(CommonConfigurationKeys.IPC_CLIENT_PING_KEY, false);
-    remoteId = ConnectionId.getConnectionId(
-        new InetSocketAddress(0), TestSaslProtocol.class, null, 0, newConf);
+    remoteId = getConnectionId(newConf);
     assertEquals(0, remoteId.getPingInterval());
   }
   


[38/46] hadoop git commit: HDFS-10173. Typo in DataXceiverServer. Contributed by Michael Han.

Posted by ae...@apache.org.
HDFS-10173. Typo in DataXceiverServer. Contributed by Michael Han.


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

Branch: refs/heads/HDFS-7240
Commit: 02a250db9f4bc54436cd9900a084215e5e3c8dae
Parents: ae14e5d
Author: Andrew Wang <wa...@apache.org>
Authored: Wed Mar 16 18:49:57 2016 -0700
Committer: Andrew Wang <wa...@apache.org>
Committed: Wed Mar 16 18:49:57 2016 -0700

----------------------------------------------------------------------
 .../org/apache/hadoop/hdfs/server/datanode/DataXceiverServer.java  | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/02a250db/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataXceiverServer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataXceiverServer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataXceiverServer.java
index 8152e6f..10945e7 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataXceiverServer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataXceiverServer.java
@@ -75,7 +75,7 @@ class DataXceiverServer implements Runnable {
     private BlockBalanceThrottler(long bandwidth, int maxThreads) {
       super(bandwidth);
       this.maxThreads.set(maxThreads);
-      LOG.info("Balancing bandwith is " + bandwidth + " bytes/s");
+      LOG.info("Balancing bandwidth is " + bandwidth + " bytes/s");
       LOG.info("Number threads for balancing is " + maxThreads);
     }
 


[24/46] hadoop git commit: YARN-4560. Make scheduler error checking message more user friendly. (Ray Chiang via kasha)

Posted by ae...@apache.org.
YARN-4560. Make scheduler error checking message more user friendly. (Ray Chiang via kasha)


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

Branch: refs/heads/HDFS-7240
Commit: 3ef5500783f043ff206458d8c10ed3d5405a8b0c
Parents: a888b5b
Author: Karthik Kambatla <ka...@apache.org>
Authored: Tue Mar 15 23:45:01 2016 -0700
Committer: Karthik Kambatla <ka...@apache.org>
Committed: Tue Mar 15 23:45:01 2016 -0700

----------------------------------------------------------------------
 .../scheduler/fair/FairScheduler.java             | 18 ++++++++++--------
 1 file changed, 10 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/3ef55007/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FairScheduler.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/fair/FairScheduler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FairScheduler.java
index ba90e21..3df0600 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FairScheduler.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FairScheduler.java
@@ -235,12 +235,13 @@ public class FairScheduler extends
 
     if (minMem < 0 || minMem > maxMem) {
       throw new YarnRuntimeException("Invalid resource scheduler memory"
-        + " allocation configuration"
-        + ", " + YarnConfiguration.RM_SCHEDULER_MINIMUM_ALLOCATION_MB
+        + " allocation configuration: "
+        + YarnConfiguration.RM_SCHEDULER_MINIMUM_ALLOCATION_MB
         + "=" + minMem
         + ", " + YarnConfiguration.RM_SCHEDULER_MAXIMUM_ALLOCATION_MB
-        + "=" + maxMem + ", min should equal greater than 0"
-        + ", max should be no smaller than min.");
+        + "=" + maxMem + ".  Both values must be greater than or equal to 0"
+        + "and the maximum allocation value must be greater than or equal to"
+        + "the minimum allocation value.");
     }
 
     // validate scheduler vcores allocation setting
@@ -253,12 +254,13 @@ public class FairScheduler extends
 
     if (minVcores < 0 || minVcores > maxVcores) {
       throw new YarnRuntimeException("Invalid resource scheduler vcores"
-        + " allocation configuration"
-        + ", " + YarnConfiguration.RM_SCHEDULER_MINIMUM_ALLOCATION_VCORES
+        + " allocation configuration: "
+        + YarnConfiguration.RM_SCHEDULER_MINIMUM_ALLOCATION_VCORES
         + "=" + minVcores
         + ", " + YarnConfiguration.RM_SCHEDULER_MAXIMUM_ALLOCATION_VCORES
-        + "=" + maxVcores + ", min should equal greater than 0"
-        + ", max should be no smaller than min.");
+        + "=" + maxVcores + ".  Both values must be greater than or equal to 0"
+          + "and the maximum allocation value must be greater than or equal to"
+          + "the minimum allocation value.");
     }
   }
 


[13/46] hadoop git commit: HDFS-9947. Block#toString should not output information from derived classes (cmccabe)

Posted by ae...@apache.org.
HDFS-9947. Block#toString should not output information from derived classes (cmccabe)


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

Branch: refs/heads/HDFS-7240
Commit: 9a43094e12ab8d35d49ceda2e2c5f83093bb3a5b
Parents: bd5556b
Author: Colin Patrick Mccabe <cm...@cloudera.com>
Authored: Mon Mar 14 14:59:11 2016 -0700
Committer: Colin Patrick Mccabe <cm...@cloudera.com>
Committed: Mon Mar 14 14:59:11 2016 -0700

----------------------------------------------------------------------
 .../src/main/java/org/apache/hadoop/hdfs/protocol/Block.java     | 4 +++-
 1 file changed, 3 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/9a43094e/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/Block.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/Block.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/Block.java
index 23bfa95..4128ece 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/Block.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/Block.java
@@ -162,7 +162,9 @@ public class Block implements Writable, Comparable<Block> {
    */
   public static String toString(final Block b) {
     StringBuilder sb = new StringBuilder();
-    b.appendStringTo(sb);
+    sb.append(BLOCK_FILE_PREFIX).
+       append(b.blockId).append("_").
+       append(b.generationStamp);
     return sb.toString();
   }
 


[39/46] hadoop git commit: MAPREDUCE-6645. TestWordStats outputs logs under directories other than target/test-dir. Contributed by Gabor Liptak.

Posted by ae...@apache.org.
MAPREDUCE-6645. TestWordStats outputs logs under directories other than target/test-dir. Contributed by Gabor Liptak.


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

Branch: refs/heads/HDFS-7240
Commit: 80fa70c4e15f16b411cb3691df546499c6a19d4c
Parents: 02a250d
Author: Akira Ajisaka <aa...@apache.org>
Authored: Thu Mar 17 18:08:31 2016 +0900
Committer: Akira Ajisaka <aa...@apache.org>
Committed: Thu Mar 17 18:08:31 2016 +0900

----------------------------------------------------------------------
 .../org/apache/hadoop/examples/TestWordStats.java    | 15 ++++++++++++---
 1 file changed, 12 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/80fa70c4/hadoop-mapreduce-project/hadoop-mapreduce-examples/src/test/java/org/apache/hadoop/examples/TestWordStats.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-examples/src/test/java/org/apache/hadoop/examples/TestWordStats.java b/hadoop-mapreduce-project/hadoop-mapreduce-examples/src/test/java/org/apache/hadoop/examples/TestWordStats.java
index 56b358e..3f0ffd6 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-examples/src/test/java/org/apache/hadoop/examples/TestWordStats.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-examples/src/test/java/org/apache/hadoop/examples/TestWordStats.java
@@ -31,15 +31,18 @@ import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.util.ToolRunner;
+import org.junit.AfterClass;
 import org.junit.Before;
 import org.junit.Test;
 
 public class TestWordStats {
 
   private final static String INPUT = "src/test/java/org/apache/hadoop/examples/pi/math";
-  private final static String MEAN_OUTPUT = "build/data/mean_output";
-  private final static String MEDIAN_OUTPUT = "build/data/median_output";
-  private final static String STDDEV_OUTPUT = "build/data/stddev_output";
+  private final static String BASEDIR = System.getProperty("test.build.data",
+                                                           "target/test-dir");
+  private final static String MEAN_OUTPUT = BASEDIR + "/mean_output";
+  private final static String MEDIAN_OUTPUT = BASEDIR + "/median_output";
+  private final static String STDDEV_OUTPUT = BASEDIR + "/stddev_output";
 
   /**
    * Modified internal test class that is designed to read all the files in the
@@ -286,4 +289,10 @@ public class TestWordStats {
     assertEquals(stddev, wr.read(INPUT), 0.0);
   }
 
+  @AfterClass public static void cleanup() throws Exception {
+    deleteDir(new File(MEAN_OUTPUT));
+    deleteDir(new File(MEDIAN_OUTPUT));
+    deleteDir(new File(STDDEV_OUTPUT));
+  }
+
 }


[41/46] hadoop git commit: YARN-4785. inconsistent value type of the type field for LeafQueueInfo in response of RM REST API.

Posted by ae...@apache.org.
YARN-4785. inconsistent value type of the type field for LeafQueueInfo in response of RM REST API.


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

Branch: refs/heads/HDFS-7240
Commit: ca8106d2dd03458944303d93679daa03b1d82ad5
Parents: f84af8b
Author: Junping Du <ju...@apache.org>
Authored: Thu Mar 17 09:04:41 2016 -0700
Committer: Junping Du <ju...@apache.org>
Committed: Thu Mar 17 09:04:41 2016 -0700

----------------------------------------------------------------------
 .../webapp/dao/CapacitySchedulerInfo.java       | 26 +++++++++++++++++---
 .../webapp/TestRMWebServicesCapacitySched.java  |  3 +++
 2 files changed, 25 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/ca8106d2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/CapacitySchedulerInfo.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/CapacitySchedulerInfo.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/CapacitySchedulerInfo.java
index f6332c1..32e4ac5 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/CapacitySchedulerInfo.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/CapacitySchedulerInfo.java
@@ -28,6 +28,9 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CSQueue;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.LeafQueue;
 
+import java.util.ArrayList;
+import java.util.List;
+
 @XmlRootElement(name = "capacityScheduler")
 @XmlType(name = "capacityScheduler")
 @XmlAccessorType(XmlAccessType.FIELD)
@@ -86,14 +89,29 @@ public class CapacitySchedulerInfo extends SchedulerInfo {
   }
 
   protected CapacitySchedulerQueueInfoList getQueues(CSQueue parent) {
-    CSQueue parentQueue = parent;
     CapacitySchedulerQueueInfoList queuesInfo =
         new CapacitySchedulerQueueInfoList();
-    for (CSQueue queue : parentQueue.getChildQueues()) {
+    // JAXB marashalling leads to situation where the "type" field injected
+    // for JSON changes from string to array depending on order of printing
+    // Issue gets fixed if all the leaf queues are marshalled before the
+    // non-leaf queues. See YARN-4785 for more details.
+    List<CSQueue> childQueues = new ArrayList<>();
+    List<CSQueue> childLeafQueues = new ArrayList<>();
+    List<CSQueue> childNonLeafQueues = new ArrayList<>();
+    for (CSQueue queue : parent.getChildQueues()) {
+      if (queue instanceof LeafQueue) {
+        childLeafQueues.add(queue);
+      } else {
+        childNonLeafQueues.add(queue);
+      }
+    }
+    childQueues.addAll(childLeafQueues);
+    childQueues.addAll(childNonLeafQueues);
+
+    for (CSQueue queue : childQueues) {
       CapacitySchedulerQueueInfo info;
       if (queue instanceof LeafQueue) {
-        info =
-            new CapacitySchedulerLeafQueueInfo((LeafQueue) queue);
+        info = new CapacitySchedulerLeafQueueInfo((LeafQueue) queue);
       } else {
         info = new CapacitySchedulerQueueInfo(queue);
         info.queues = getQueues(queue);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ca8106d2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesCapacitySched.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/TestRMWebServicesCapacitySched.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesCapacitySched.java
index 28b1c4f..12f0c8b 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesCapacitySched.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesCapacitySched.java
@@ -43,6 +43,7 @@ import org.apache.hadoop.yarn.webapp.WebServicesTestUtils;
 import org.codehaus.jettison.json.JSONArray;
 import org.codehaus.jettison.json.JSONException;
 import org.codehaus.jettison.json.JSONObject;
+import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
 import org.w3c.dom.Document;
@@ -379,6 +380,8 @@ public class TestRMWebServicesCapacitySched extends JerseyTestBase {
         verifySubQueue(obj, q2, qi.absoluteCapacity, qi.absoluteMaxCapacity);
       }
     } else {
+      Assert.assertEquals("\"type\" field is incorrect",
+          "capacitySchedulerLeafQueueInfo", info.getString("type"));
       LeafQueueInfo lqi = (LeafQueueInfo) qi;
       lqi.numActiveApplications = info.getInt("numActiveApplications");
       lqi.numPendingApplications = info.getInt("numPendingApplications");


[34/46] hadoop git commit: Revert "CapacityScheduler: Improve preemption to only kill containers that would satisfy the incoming request. (Wangda Tan)"

Posted by ae...@apache.org.
Revert "CapacityScheduler: Improve preemption to only kill containers that would satisfy the incoming request. (Wangda Tan)"

This reverts commit 7e8c9beb4156dcaeb3a11e60aaa06d2370626913.


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

Branch: refs/heads/HDFS-7240
Commit: fa7a43529d529f0006c8033c2003f15b9b93f103
Parents: 7e8c9be
Author: Wangda Tan <wa...@apache.org>
Authored: Wed Mar 16 17:02:10 2016 -0700
Committer: Wangda Tan <wa...@apache.org>
Committed: Wed Mar 16 17:02:10 2016 -0700

----------------------------------------------------------------------
 .../ProportionalCapacityPreemptionPolicy.java   | 166 ++---
 .../rmcontainer/RMContainer.java                |   1 -
 .../scheduler/PreemptableResourceScheduler.java |   2 +-
 .../scheduler/ResourceLimits.java               |   9 -
 .../scheduler/SchedulerNode.java                |   9 +-
 .../scheduler/capacity/AbstractCSQueue.java     |  45 +-
 .../scheduler/capacity/CSAssignment.java        |  11 -
 .../scheduler/capacity/CapacityScheduler.java   | 132 +---
 .../CapacitySchedulerConfiguration.java         |  14 +-
 .../capacity/CapacitySchedulerContext.java      |  15 +-
 .../scheduler/capacity/LeafQueue.java           |  69 --
 .../scheduler/capacity/ParentQueue.java         | 157 +----
 .../allocator/AbstractContainerAllocator.java   |   2 -
 .../capacity/allocator/ContainerAllocation.java |  12 -
 .../allocator/RegularContainerAllocator.java    |  39 +-
 .../capacity/preemption/KillableContainer.java  |  45 --
 .../capacity/preemption/PreemptableQueue.java   | 102 ---
 .../capacity/preemption/PreemptionManager.java  | 165 -----
 .../scheduler/common/AssignmentInformation.java |   6 +-
 .../scheduler/common/fica/FiCaSchedulerApp.java |  32 +-
 .../common/fica/FiCaSchedulerNode.java          |  65 +-
 .../scheduler/event/SchedulerEventType.java     |  15 +-
 .../resourcemanager/TestRMDispatcher.java       |   4 +-
 .../server/resourcemanager/TestRMRestart.java   |   2 +-
 .../applicationsmanager/TestAMRestart.java      |   7 +-
 ...estProportionalCapacityPreemptionPolicy.java |   6 +-
 ...pacityPreemptionPolicyForNodePartitions.java |   2 -
 .../capacity/TestApplicationLimits.java         |   2 -
 .../capacity/TestApplicationPriority.java       |   6 +-
 .../capacity/TestCapacityScheduler.java         |   8 +-
 .../TestCapacitySchedulerPreemption.java        | 677 -------------------
 .../scheduler/capacity/TestChildQueueOrder.java |   2 -
 .../scheduler/capacity/TestLeafQueue.java       |   3 -
 .../TestNodeLabelContainerAllocation.java       |  97 ---
 .../scheduler/capacity/TestParentQueue.java     |   2 -
 .../scheduler/capacity/TestReservations.java    |   2 -
 .../scheduler/capacity/TestUtils.java           |  36 -
 .../fair/TestFairSchedulerPreemption.java       |   2 +-
 38 files changed, 186 insertions(+), 1785 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/fa7a4352/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/ProportionalCapacityPreemptionPolicy.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/monitor/capacity/ProportionalCapacityPreemptionPolicy.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/ProportionalCapacityPreemptionPolicy.java
index 9b499c8..3a87edb 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/ProportionalCapacityPreemptionPolicy.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/ProportionalCapacityPreemptionPolicy.java
@@ -35,7 +35,7 @@ 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.ApplicationAttemptId;
-import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.Priority;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
@@ -49,7 +49,6 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.Capacity
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.LeafQueue;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueueCapacities;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.preemption.PreemptableQueue;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerApp;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.ContainerPreemptEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEventType;
@@ -126,8 +125,8 @@ public class ProportionalCapacityPreemptionPolicy implements SchedulingEditPolic
   private long maxWaitTime;
   private CapacityScheduler scheduler;
   private long monitoringInterval;
-  private final Map<RMContainer, Long> preempted = new HashMap<>();
-
+  private final Map<RMContainer,Long> preempted =
+    new HashMap<RMContainer,Long>();
   private ResourceCalculator rc;
   private float percentageClusterPreemptionAllowed;
   private double naturalTerminationFactor;
@@ -136,10 +135,6 @@ public class ProportionalCapacityPreemptionPolicy implements SchedulingEditPolic
       new HashMap<>();
   private RMNodeLabelsManager nlm;
 
-  // Preemptable Entities, synced from scheduler at every run
-  private Map<String, PreemptableQueue> preemptableEntities = null;
-  private Set<ContainerId> killableContainers;
-
   public ProportionalCapacityPreemptionPolicy() {
     clock = SystemClock.getInstance();
   }
@@ -189,64 +184,6 @@ public class ProportionalCapacityPreemptionPolicy implements SchedulingEditPolic
     Resource clusterResources = Resources.clone(scheduler.getClusterResource());
     containerBasedPreemptOrKill(root, clusterResources);
   }
-
-  @SuppressWarnings("unchecked")
-  private void cleanupStaledKillableContainers(Resource cluster,
-      Set<String> leafQueueNames) {
-    for (String q : leafQueueNames) {
-      for (TempQueuePerPartition tq : getQueuePartitions(q)) {
-        // When queue's used - killable <= guaranteed and, killable > 0, we need
-        // to check if any of killable containers needs to be reverted
-        if (Resources.lessThanOrEqual(rc, cluster,
-            Resources.subtract(tq.current, tq.killable), tq.idealAssigned)
-            && Resources.greaterThan(rc, cluster, tq.killable, Resources.none())) {
-          // How many killable resources need to be reverted
-          // need-to-revert = already-marked-killable - (current - ideal)
-          Resource toBeRevertedFromKillable = Resources.subtract(tq.killable,
-              Resources.subtract(tq.current, tq.idealAssigned));
-
-          Resource alreadyReverted = Resources.createResource(0);
-
-          for (RMContainer c : preemptableEntities.get(q).getKillableContainers(
-              tq.partition).values()) {
-            if (Resources.greaterThanOrEqual(rc, cluster, alreadyReverted,
-                toBeRevertedFromKillable)) {
-              break;
-            }
-
-            if (Resources.greaterThan(rc, cluster,
-                Resources.add(alreadyReverted, c.getAllocatedResource()),
-                toBeRevertedFromKillable)) {
-              continue;
-            } else {
-              // This container need to be marked to unkillable
-              Resources.addTo(alreadyReverted, c.getAllocatedResource());
-              rmContext.getDispatcher().getEventHandler().handle(
-                  new ContainerPreemptEvent(c.getApplicationAttemptId(), c,
-                      SchedulerEventType.MARK_CONTAINER_FOR_NONKILLABLE));
-            }
-          }
-
-        }
-      }
-    }
-  }
-
-  private void syncKillableContainersFromScheduler() {
-    // sync preemptable entities from scheduler
-    preemptableEntities =
-        scheduler.getPreemptionManager().getShallowCopyOfPreemptableEntities();
-
-    killableContainers = new HashSet<>();
-    for (Map.Entry<String, PreemptableQueue> entry : preemptableEntities
-        .entrySet()) {
-      PreemptableQueue entity = entry.getValue();
-      for (Map<ContainerId, RMContainer> map : entity.getKillableContainers()
-          .values()) {
-        killableContainers.addAll(map.keySet());
-      }
-    }
-  }
   
   /**
    * This method selects and tracks containers to be preempted. If a container
@@ -264,8 +201,6 @@ public class ProportionalCapacityPreemptionPolicy implements SchedulingEditPolic
         .getNodeLabelManager().getClusterNodeLabelNames());
     allPartitions.add(RMNodeLabelsManager.NO_LABEL);
 
-    syncKillableContainersFromScheduler();
-
     // extract a summary of the queues from scheduler
     synchronized (scheduler) {
       queueToPartitions.clear();
@@ -293,17 +228,13 @@ public class ProportionalCapacityPreemptionPolicy implements SchedulingEditPolic
           recursivelyComputeIdealAssignment(tRoot, totalPreemptionAllowed);
     }
 
-    // remove containers from killable list when we want to preempt less resources
-    // from queue.
-    cleanupStaledKillableContainers(clusterResources, leafQueueNames);
-
     // based on ideal allocation select containers to be preempted from each
     // queue and each application
     Map<ApplicationAttemptId,Set<RMContainer>> toPreempt =
         getContainersToPreempt(leafQueueNames, clusterResources);
 
     if (LOG.isDebugEnabled()) {
-      logToCSV(new ArrayList<>(leafQueueNames));
+      logToCSV(new ArrayList<String>(leafQueueNames));
     }
 
     // if we are in observeOnly mode return before any action is taken
@@ -323,10 +254,10 @@ public class ProportionalCapacityPreemptionPolicy implements SchedulingEditPolic
         // if we tried to preempt this for more than maxWaitTime
         if (preempted.get(container) != null &&
             preempted.get(container) + maxWaitTime < clock.getTime()) {
-          // mark container killable
+          // kill it
           rmContext.getDispatcher().getEventHandler().handle(
               new ContainerPreemptEvent(appAttemptId, container,
-                  SchedulerEventType.MARK_CONTAINER_FOR_KILLABLE));
+                  SchedulerEventType.KILL_PREEMPTED_CONTAINER));
           preempted.remove(container);
         } else {
           if (preempted.get(container) != null) {
@@ -402,14 +333,14 @@ public class ProportionalCapacityPreemptionPolicy implements SchedulingEditPolic
 
     // qAlloc tracks currently active queues (will decrease progressively as
     // demand is met)
-    List<TempQueuePerPartition> qAlloc = new ArrayList<>(queues);
+    List<TempQueuePerPartition> qAlloc = new ArrayList<TempQueuePerPartition>(queues);
     // unassigned tracks how much resources are still to assign, initialized
     // with the total capacity for this set of queues
     Resource unassigned = Resources.clone(tot_guarant);
 
     // group queues based on whether they have non-zero guaranteed capacity
-    Set<TempQueuePerPartition> nonZeroGuarQueues = new HashSet<>();
-    Set<TempQueuePerPartition> zeroGuarQueues = new HashSet<>();
+    Set<TempQueuePerPartition> nonZeroGuarQueues = new HashSet<TempQueuePerPartition>();
+    Set<TempQueuePerPartition> zeroGuarQueues = new HashSet<TempQueuePerPartition>();
 
     for (TempQueuePerPartition q : qAlloc) {
       if (Resources
@@ -484,8 +415,8 @@ public class ProportionalCapacityPreemptionPolicy implements SchedulingEditPolic
     // idealAssigned >= current + pending), remove it from consideration.
     // Sort queues from most under-guaranteed to most over-guaranteed.
     TQComparator tqComparator = new TQComparator(rc, tot_guarant);
-    PriorityQueue<TempQueuePerPartition> orderedByNeed = new PriorityQueue<>(10,
-        tqComparator);
+    PriorityQueue<TempQueuePerPartition> orderedByNeed =
+        new PriorityQueue<TempQueuePerPartition>(10, tqComparator);
     for (Iterator<TempQueuePerPartition> i = qAlloc.iterator(); i.hasNext();) {
       TempQueuePerPartition q = i.next();
       if (Resources.greaterThan(rc, tot_guarant, q.current, q.guaranteed)) {
@@ -543,7 +474,7 @@ public class ProportionalCapacityPreemptionPolicy implements SchedulingEditPolic
   // percentage of guaranteed.
   protected Collection<TempQueuePerPartition> getMostUnderservedQueues(
       PriorityQueue<TempQueuePerPartition> orderedByNeed, TQComparator tqComparator) {
-    ArrayList<TempQueuePerPartition> underserved = new ArrayList<>();
+    ArrayList<TempQueuePerPartition> underserved = new ArrayList<TempQueuePerPartition>();
     while (!orderedByNeed.isEmpty()) {
       TempQueuePerPartition q1 = orderedByNeed.remove();
       underserved.add(q1);
@@ -571,7 +502,7 @@ public class ProportionalCapacityPreemptionPolicy implements SchedulingEditPolic
     
     if (ignoreGuar) {
       for (TempQueuePerPartition q : queues) {
-        q.normalizedGuarantee = 1.0f / queues.size();
+        q.normalizedGuarantee = (float)  1.0f / ((float) queues.size());
       }
     } else {
       for (TempQueuePerPartition q : queues) {
@@ -584,9 +515,8 @@ public class ProportionalCapacityPreemptionPolicy implements SchedulingEditPolic
     }
   }
 
-  private String getPartitionByRMContainer(RMContainer rmContainer) {
-    return scheduler.getSchedulerNode(rmContainer.getAllocatedNode())
-        .getPartition();
+  private String getPartitionByNodeId(NodeId nodeId) {
+    return scheduler.getSchedulerNode(nodeId).getPartition();
   }
 
   /**
@@ -604,7 +534,7 @@ public class ProportionalCapacityPreemptionPolicy implements SchedulingEditPolic
       return false;
     }
 
-    String nodePartition = getPartitionByRMContainer(rmContainer);
+    String nodePartition = getPartitionByNodeId(rmContainer.getAllocatedNode());
     Resource toObtainByPartition =
         resourceToObtainByPartitions.get(nodePartition);
 
@@ -645,7 +575,7 @@ public class ProportionalCapacityPreemptionPolicy implements SchedulingEditPolic
       ApplicationAttemptId appAttemptId, RMContainer containerToPreempt) {
     Set<RMContainer> set;
     if (null == (set = preemptMap.get(appAttemptId))) {
-      set = new HashSet<>();
+      set = new HashSet<RMContainer>();
       preemptMap.put(appAttemptId, set);
     }
     set.add(containerToPreempt);
@@ -657,7 +587,7 @@ public class ProportionalCapacityPreemptionPolicy implements SchedulingEditPolic
    * over-capacity queue. It uses {@link #NATURAL_TERMINATION_FACTOR} to
    * account for containers that will naturally complete.
    *
-   * @param leafQueueNames set of leaf queues to preempt from
+   * @param queues set of leaf queues to preempt from
    * @param clusterResource total amount of cluster resources
    * @return a map of applciationID to set of containers to preempt
    */
@@ -665,8 +595,8 @@ public class ProportionalCapacityPreemptionPolicy implements SchedulingEditPolic
       Set<String> leafQueueNames, Resource clusterResource) {
 
     Map<ApplicationAttemptId, Set<RMContainer>> preemptMap =
-        new HashMap<>();
-    List<RMContainer> skippedAMContainerlist = new ArrayList<>();
+        new HashMap<ApplicationAttemptId, Set<RMContainer>>();
+    List<RMContainer> skippedAMContainerlist = new ArrayList<RMContainer>();
 
     // Loop all leaf queues
     for (String queueName : leafQueueNames) {
@@ -684,7 +614,7 @@ public class ProportionalCapacityPreemptionPolicy implements SchedulingEditPolic
       LeafQueue leafQueue = null;
 
       Map<String, Resource> resToObtainByPartition =
-          new HashMap<>();
+          new HashMap<String, Resource>();
       for (TempQueuePerPartition qT : getQueuePartitions(queueName)) {
         leafQueue = qT.leafQueue;
         // we act only if we are violating balance by more than
@@ -773,6 +703,7 @@ public class ProportionalCapacityPreemptionPolicy implements SchedulingEditPolic
    * @param clusterResource
    * @param preemptMap
    * @param skippedAMContainerlist
+   * @param resToObtain
    * @param skippedAMSize
    * @param maxAMCapacityForThisQueue
    */
@@ -820,7 +751,7 @@ public class ProportionalCapacityPreemptionPolicy implements SchedulingEditPolic
 
     // first drop reserved containers towards rsrcPreempt
     List<RMContainer> reservedContainers =
-        new ArrayList<>(app.getReservedContainers());
+        new ArrayList<RMContainer>(app.getReservedContainers());
     for (RMContainer c : reservedContainers) {
       if (resToObtainByPartition.isEmpty()) {
         return;
@@ -840,7 +771,8 @@ public class ProportionalCapacityPreemptionPolicy implements SchedulingEditPolic
     // if more resources are to be freed go through all live containers in
     // reverse priority and reverse allocation order and mark them for
     // preemption
-    List<RMContainer> liveContainers = new ArrayList<>(app.getLiveContainers());
+    List<RMContainer> liveContainers =
+      new ArrayList<RMContainer>(app.getLiveContainers());
 
     sortContainers(liveContainers);
 
@@ -856,11 +788,6 @@ public class ProportionalCapacityPreemptionPolicy implements SchedulingEditPolic
         continue;
       }
 
-      // Skip already marked to killable containers
-      if (killableContainers.contains(c.getContainerId())) {
-        continue;
-      }
-
       // Try to preempt this container
       tryPreemptContainerAndDeductResToObtain(resToObtainByPartition, c,
           clusterResource, preemptMap);
@@ -899,10 +826,6 @@ public class ProportionalCapacityPreemptionPolicy implements SchedulingEditPolic
     return "ProportionalCapacityPreemptionPolicy";
   }
 
-  @VisibleForTesting
-  public Map<RMContainer, Long> getToPreemptContainers() {
-    return preempted;
-  }
 
   /**
    * This method walks a tree of CSQueue and clones the portion of the state
@@ -928,11 +851,6 @@ public class ProportionalCapacityPreemptionPolicy implements SchedulingEditPolic
           partitionToLookAt);
       Resource guaranteed = Resources.multiply(partitionResource, absCap);
       Resource maxCapacity = Resources.multiply(partitionResource, absMaxCap);
-      Resource killable = Resources.none();
-      if (null != preemptableEntities.get(queueName)) {
-         killable = preemptableEntities.get(queueName)
-            .getKillableResource(partitionToLookAt);
-      }
 
       // when partition is a non-exclusive partition, the actual maxCapacity
       // could more than specified maxCapacity
@@ -957,7 +875,7 @@ public class ProportionalCapacityPreemptionPolicy implements SchedulingEditPolic
               l.getTotalPendingResourcesConsideringUserLimit(
                   partitionResource, partitionToLookAt);
         ret = new TempQueuePerPartition(queueName, current, pending, guaranteed,
-            maxCapacity, preemptionDisabled, partitionToLookAt, killable);
+            maxCapacity, preemptionDisabled, partitionToLookAt);
         if (preemptionDisabled) {
           ret.untouchableExtra = extra;
         } else {
@@ -968,7 +886,7 @@ public class ProportionalCapacityPreemptionPolicy implements SchedulingEditPolic
         Resource pending = Resource.newInstance(0, 0);
         ret =
             new TempQueuePerPartition(curQueue.getQueueName(), current, pending,
-                guaranteed, maxCapacity, false, partitionToLookAt, killable);
+                guaranteed, maxCapacity, false, partitionToLookAt);
         Resource childrensPreemptable = Resource.newInstance(0, 0);
         for (CSQueue c : curQueue.getChildQueues()) {
           TempQueuePerPartition subq =
@@ -1014,7 +932,7 @@ public class ProportionalCapacityPreemptionPolicy implements SchedulingEditPolic
 
     Map<String, TempQueuePerPartition> queuePartitions;
     if (null == (queuePartitions = queueToPartitions.get(queueName))) {
-      queuePartitions = new HashMap<>();
+      queuePartitions = new HashMap<String, TempQueuePerPartition>();
       queueToPartitions.put(queueName, queuePartitions);
     }
     queuePartitions.put(queuePartition.partition, queuePartition);
@@ -1053,10 +971,8 @@ public class ProportionalCapacityPreemptionPolicy implements SchedulingEditPolic
     final Resource guaranteed;
     final Resource maxCapacity;
     final String partition;
-    final Resource killable;
     Resource idealAssigned;
     Resource toBePreempted;
-
     // For logging purpose
     Resource actuallyPreempted;
     Resource untouchableExtra;
@@ -1070,7 +986,7 @@ public class ProportionalCapacityPreemptionPolicy implements SchedulingEditPolic
 
     TempQueuePerPartition(String queueName, Resource current, Resource pending,
         Resource guaranteed, Resource maxCapacity, boolean preemptionDisabled,
-        String partition, Resource killableResource) {
+        String partition) {
       this.queueName = queueName;
       this.current = current;
       this.pending = pending;
@@ -1080,12 +996,11 @@ public class ProportionalCapacityPreemptionPolicy implements SchedulingEditPolic
       this.actuallyPreempted = Resource.newInstance(0, 0);
       this.toBePreempted = Resource.newInstance(0, 0);
       this.normalizedGuarantee = Float.NaN;
-      this.children = new ArrayList<>();
+      this.children = new ArrayList<TempQueuePerPartition>();
       this.untouchableExtra = Resource.newInstance(0, 0);
       this.preemptableExtra = Resource.newInstance(0, 0);
       this.preemptionDisabled = preemptionDisabled;
       this.partition = partition;
-      this.killable = killableResource;
     }
 
     public void setLeafQueue(LeafQueue l){
@@ -1103,6 +1018,12 @@ public class ProportionalCapacityPreemptionPolicy implements SchedulingEditPolic
       Resources.addTo(pending, q.pending);
     }
 
+    public void addChildren(ArrayList<TempQueuePerPartition> queues) {
+      assert leafQueue == null;
+      children.addAll(queues);
+    }
+
+
     public ArrayList<TempQueuePerPartition> getChildren(){
       return children;
     }
@@ -1143,13 +1064,18 @@ public class ProportionalCapacityPreemptionPolicy implements SchedulingEditPolic
       return sb.toString();
     }
 
+    public void printAll() {
+      LOG.info(this.toString());
+      for (TempQueuePerPartition sub : this.getChildren()) {
+        sub.printAll();
+      }
+    }
+
     public void assignPreemption(float scalingFactor,
         ResourceCalculator rc, Resource clusterResource) {
-      if (Resources.greaterThan(rc, clusterResource,
-          Resources.subtract(current, killable), idealAssigned)) {
-        toBePreempted = Resources.multiply(Resources.subtract(
-            Resources.subtract(current, killable), idealAssigned),
-            scalingFactor);
+      if (Resources.greaterThan(rc, clusterResource, current, idealAssigned)) {
+          toBePreempted = Resources.multiply(
+              Resources.subtract(current, idealAssigned), scalingFactor);
       } else {
         toBePreempted = Resource.newInstance(0, 0);
       }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fa7a4352/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 dfe0886..5d26931 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
@@ -30,7 +30,6 @@ import org.apache.hadoop.yarn.api.records.Priority;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.ResourceRequest;
 import org.apache.hadoop.yarn.event.EventHandler;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Queue;
 
 /**
  * Represents the ResourceManager's view of an application container. See 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fa7a4352/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/PreemptableResourceScheduler.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/PreemptableResourceScheduler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/PreemptableResourceScheduler.java
index b73c538..ee7e101 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/PreemptableResourceScheduler.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/PreemptableResourceScheduler.java
@@ -45,6 +45,6 @@ public interface PreemptableResourceScheduler extends ResourceScheduler {
    * Ask the scheduler to forcibly interrupt the container given as input
    * @param container
    */
-  void markContainerForKillable(RMContainer container);
+  void killPreemptedContainer(RMContainer container);
 
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fa7a4352/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/ResourceLimits.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/ResourceLimits.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/ResourceLimits.java
index 721eb36..c545e9e 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/ResourceLimits.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/ResourceLimits.java
@@ -38,8 +38,6 @@ public class ResourceLimits {
   // containers.
   private volatile Resource headroom;
 
-  private boolean allowPreempt = false;
-
   public ResourceLimits(Resource limit) {
     this(limit, Resources.none());
   }
@@ -74,11 +72,4 @@ public class ResourceLimits {
     this.amountNeededUnreserve = amountNeededUnreserve;
   }
 
-  public boolean isAllowPreemption() {
-    return allowPreempt;
-  }
-
-  public void setIsAllowPreemption(boolean allowPreempt) {
-   this.allowPreempt = allowPreempt;
-  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fa7a4352/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerNode.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/SchedulerNode.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerNode.java
index 6c4f300..33ab2f1 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerNode.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerNode.java
@@ -64,8 +64,9 @@ public abstract class SchedulerNode {
   private volatile ResourceUtilization nodeUtilization =
       ResourceUtilization.newInstance(0, 0, 0f);
 
-  /* set of containers that are allocated containers */
-  protected final Map<ContainerId, RMContainer> launchedContainers =
+
+  /** Set of containers that are allocated containers. */
+  private final Map<ContainerId, RMContainer> launchedContainers =
       new HashMap<>();
 
   private final RMNode rmNode;
@@ -167,7 +168,7 @@ public abstract class SchedulerNode {
    * @param deltaResource Change in the resource allocation.
    * @param increase True if the change is an increase of allocation.
    */
-  protected synchronized void changeContainerResource(ContainerId containerId,
+  private synchronized void changeContainerResource(ContainerId containerId,
       Resource deltaResource, boolean increase) {
     if (increase) {
       deductUnallocatedResource(deltaResource);
@@ -241,7 +242,7 @@ public abstract class SchedulerNode {
    * Update the resources of the node when allocating a new container.
    * @param container Container to allocate.
    */
-  protected synchronized void updateResource(Container container) {
+  private synchronized void updateResource(Container container) {
     addUnallocatedResource(container.getResource());
     --numContainers;
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fa7a4352/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 955f8fa..39ca29b 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
@@ -21,7 +21,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.Iterator;
 import java.util.Map;
 import java.util.Set;
 
@@ -46,7 +45,6 @@ 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.rmcontainer.RMContainer;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceLimits;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceUsage;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplicationAttempt;
@@ -442,8 +440,11 @@ public abstract class AbstractCSQueue implements CSQueue {
           Resources.multiplyAndNormalizeDown(resourceCalculator,
               labelManager.getResourceByLabel(nodePartition, clusterResource),
               queueCapacities.getAbsoluteMaximumCapacity(nodePartition), minimumAllocation);
-      return Resources.min(resourceCalculator, clusterResource,
-          queueMaxResource, currentResourceLimits.getLimit());
+      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.
@@ -473,19 +474,12 @@ public abstract class AbstractCSQueue implements CSQueue {
 
     Resource nowTotalUsed = queueUsage.getUsed(nodePartition);
 
-    // Set headroom for currentResourceLimits:
-    // When queue is a parent queue: Headroom = limit - used + killable
-    // When queue is a leaf queue: Headroom = limit - used (leaf queue cannot preempt itself)
-    Resource usedExceptKillable = nowTotalUsed;
-    if (null != getChildQueues() && !getChildQueues().isEmpty()) {
-      usedExceptKillable = Resources.subtract(nowTotalUsed,
-          getTotalKillableResource(nodePartition));
-    }
-    currentResourceLimits.setHeadroom(
-        Resources.subtract(currentLimitResource, usedExceptKillable));
+    // Set headroom for currentResourceLimits
+    currentResourceLimits.setHeadroom(Resources.subtract(currentLimitResource,
+        nowTotalUsed));
 
     if (Resources.greaterThanOrEqual(resourceCalculator, clusterResource,
-        usedExceptKillable, currentLimitResource)) {
+        nowTotalUsed, currentLimitResource)) {
 
       // if reservation continous looking enabled, check to see if could we
       // potentially use this node instead of a reserved node if the application
@@ -497,7 +491,7 @@ public abstract class AbstractCSQueue implements CSQueue {
               resourceCouldBeUnreserved, Resources.none())) {
         // resource-without-reserved = used - reserved
         Resource newTotalWithoutReservedResource =
-            Resources.subtract(usedExceptKillable, resourceCouldBeUnreserved);
+            Resources.subtract(nowTotalUsed, resourceCouldBeUnreserved);
 
         // when total-used-without-reserved-resource < currentLimit, we still
         // have chance to allocate on this node by unreserving some containers
@@ -626,10 +620,11 @@ public abstract class AbstractCSQueue implements CSQueue {
     // considering all labels in cluster, only those labels which are
     // use some resource of this queue can be considered.
     Set<String> nodeLabels = new HashSet<String>();
-    if (this.getAccessibleNodeLabels() != null && this.getAccessibleNodeLabels()
-        .contains(RMNodeLabelsManager.ANY)) {
-      nodeLabels.addAll(Sets.union(this.getQueueCapacities().getNodePartitionsSet(),
-          this.getQueueResourceUsage().getNodePartitionsSet()));
+    if (this.getAccessibleNodeLabels() != null
+        && this.getAccessibleNodeLabels().contains(RMNodeLabelsManager.ANY)) {
+      nodeLabels.addAll(Sets.union(this.getQueueCapacities()
+          .getNodePartitionsSet(), this.getQueueResourceUsage()
+          .getNodePartitionsSet()));
     } else {
       nodeLabels.addAll(this.getAccessibleNodeLabels());
     }
@@ -641,14 +636,4 @@ public abstract class AbstractCSQueue implements CSQueue {
     }
     return nodeLabels;
   }
-
-  public Resource getTotalKillableResource(String partition) {
-    return csContext.getPreemptionManager().getKillableResource(queueName,
-        partition);
-  }
-
-  public Iterator<RMContainer> getKillableContainers(String partition) {
-    return csContext.getPreemptionManager().getKillableContainers(queueName,
-        partition);
-  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fa7a4352/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CSAssignment.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/CSAssignment.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/CSAssignment.java
index 6406efe..68f6f12 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/CSAssignment.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/CSAssignment.java
@@ -26,8 +26,6 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.Assignment
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerApp;
 import org.apache.hadoop.yarn.util.resource.Resources;
 
-import java.util.List;
-
 @Private
 @Unstable
 public class CSAssignment {
@@ -44,7 +42,6 @@ public class CSAssignment {
   private boolean fulfilledReservation;
   private final AssignmentInformation assignmentInformation;
   private boolean increaseAllocation;
-  private List<RMContainer> containersToKill;
 
   public CSAssignment(Resource resource, NodeType type) {
     this(resource, type, null, null, false, false);
@@ -150,12 +147,4 @@ public class CSAssignment {
   public void setIncreasedAllocation(boolean flag) {
     increaseAllocation = flag;
   }
-
-  public void setContainersToKill(List<RMContainer> containersToKill) {
-    this.containersToKill = containersToKill;
-  }
-
-  public List<RMContainer> getContainersToKill() {
-    return containersToKill;
-  }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fa7a4352/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 cf5c3b5..735306a 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
@@ -108,8 +108,6 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerDynamicE
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerHealth;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNode;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerUtils;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.preemption.KillableContainer;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.preemption.PreemptionManager;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.AssignmentInformation;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.QueueEntitlement;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerApp;
@@ -150,10 +148,6 @@ public class CapacityScheduler extends
   // timeout to join when we stop this service
   protected final long THREAD_JOIN_TIMEOUT_MS = 1000;
 
-  private PreemptionManager preemptionManager = new PreemptionManager();
-
-  private volatile boolean isLazyPreemptionEnabled = false;
-
   static final Comparator<CSQueue> nonPartitionedQueueComparator =
       new Comparator<CSQueue>() {
     @Override
@@ -304,11 +298,12 @@ public class CapacityScheduler extends
     initMaximumResourceCapability(this.conf.getMaximumAllocation());
     this.calculator = this.conf.getResourceCalculator();
     this.usePortForNodeName = this.conf.getUsePortForNodeName();
-    this.applications = new ConcurrentHashMap<>();
+    this.applications =
+        new ConcurrentHashMap<ApplicationId,
+            SchedulerApplication<FiCaSchedulerApp>>();
     this.labelManager = rmContext.getNodeLabelManager();
     authorizer = YarnAuthorizationProvider.getInstance(yarnConf);
     initializeQueues(this.conf);
-    this.isLazyPreemptionEnabled = conf.getLazyPreemptionEnabled();
 
     scheduleAsynchronously = this.conf.getScheduleAynschronously();
     asyncScheduleInterval =
@@ -374,9 +369,6 @@ public class CapacityScheduler extends
       refreshMaximumAllocation(this.conf.getMaximumAllocation());
       throw new IOException("Failed to re-init queues", t);
     }
-
-    // update lazy preemption
-    this.isLazyPreemptionEnabled = this.conf.getLazyPreemptionEnabled();
   }
   
   long getAsyncScheduleInterval() {
@@ -511,9 +503,6 @@ public class CapacityScheduler extends
     LOG.info("Initialized root queue " + root);
     updatePlacementRules();
     setQueueAcls(authorizer, queues);
-
-    // Notify Preemption Manager
-    preemptionManager.refreshQueues(null, root);
   }
 
   @Lock(CapacityScheduler.class)
@@ -542,9 +531,6 @@ public class CapacityScheduler extends
 
     labelManager.reinitializeQueueLabels(getQueueToLabels());
     setQueueAcls(authorizer, queues);
-
-    // Notify Preemption Manager
-    preemptionManager.refreshQueues(null, root);
   }
 
   @VisibleForTesting
@@ -1267,10 +1253,8 @@ public class CapacityScheduler extends
 
     // Try to schedule more if there are no reservations to fulfill
     if (node.getReservedContainer() == null) {
-      if (calculator.computeAvailableContainers(Resources
-              .add(node.getUnallocatedResource(), node.getTotalKillableResources()),
-          minimumAllocation) > 0) {
-
+      if (calculator.computeAvailableContainers(node.getUnallocatedResource(),
+        minimumAllocation) > 0) {
         if (LOG.isDebugEnabled()) {
           LOG.debug("Trying to schedule on node: " + node.getNodeName() +
               ", available: " + node.getUnallocatedResource());
@@ -1279,8 +1263,10 @@ public class CapacityScheduler extends
         assignment = root.assignContainers(
             getClusterResource(),
             node,
+            // TODO, now we only consider limits for parent for non-labeled
+            // resources, should consider labeled resources as well.
             new ResourceLimits(labelManager.getResourceByLabel(
-                node.getPartition(), getClusterResource())),
+                RMNodeLabelsManager.NO_LABEL, getClusterResource())),
             SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
         if (Resources.greaterThan(calculator, getClusterResource(),
             assignment.getResource(), Resources.none())) {
@@ -1450,20 +1436,11 @@ public class CapacityScheduler extends
       markContainerForPreemption(aid, containerToBePreempted);
     }
     break;
-    case MARK_CONTAINER_FOR_KILLABLE:
-    {
-      ContainerPreemptEvent containerKillableEvent = (ContainerPreemptEvent)event;
-      RMContainer killableContainer = containerKillableEvent.getContainer();
-      markContainerForKillable(killableContainer);
-    }
-    break;
-    case MARK_CONTAINER_FOR_NONKILLABLE:
+    case KILL_PREEMPTED_CONTAINER:
     {
-      if (isLazyPreemptionEnabled) {
-        ContainerPreemptEvent cancelKillContainerEvent =
-            (ContainerPreemptEvent) event;
-        markContainerForNonKillable(cancelKillContainerEvent.getContainer());
-      }
+      ContainerPreemptEvent killContainerEvent = (ContainerPreemptEvent)event;
+      RMContainer containerToBeKilled = killContainerEvent.getContainer();
+      killPreemptedContainer(containerToBeKilled);
     }
     break;
     default:
@@ -1571,14 +1548,14 @@ public class CapacityScheduler extends
   protected void completedContainerInternal(
       RMContainer rmContainer, ContainerStatus containerStatus,
       RMContainerEventType event) {
+    
     Container container = rmContainer.getContainer();
-    ContainerId containerId = container.getId();
     
     // Get the application for the finished container
     FiCaSchedulerApp application =
         getCurrentAttemptForContainer(container.getId());
     ApplicationId appId =
-        containerId.getApplicationAttemptId().getApplicationId();
+        container.getId().getApplicationAttemptId().getApplicationId();
     if (application == null) {
       LOG.info("Container " + container + " of" + " finished application "
           + appId + " completed with event " + event);
@@ -1592,6 +1569,15 @@ public class CapacityScheduler extends
     LeafQueue queue = (LeafQueue)application.getQueue();
     queue.completedContainer(getClusterResource(), application, node,
         rmContainer, containerStatus, event, null, true);
+
+    if (containerStatus.getExitStatus() == ContainerExitStatus.PREEMPTED) {
+      schedulerHealth.updatePreemption(Time.now(), container.getNodeId(),
+        container.getId(), queue.getQueuePath());
+      schedulerHealth.updateSchedulerPreemptionCounts(1);
+    } else {
+      schedulerHealth.updateRelease(lastNodeUpdateTime, container.getNodeId(),
+        container.getId(), queue.getQueuePath());
+    }
   }
   
   @Override
@@ -1627,7 +1613,7 @@ public class CapacityScheduler extends
       ApplicationAttemptId applicationAttemptId) {
     return super.getApplicationAttempt(applicationAttemptId);
   }
-
+  
   @Lock(Lock.NoLock.class)
   public FiCaSchedulerNode getNode(NodeId nodeId) {
     return nodeTracker.getNode(nodeId);
@@ -1668,60 +1654,15 @@ public class CapacityScheduler extends
     }
   }
 
-  public synchronized void markContainerForKillable(
-      RMContainer killableContainer) {
-    if (LOG.isDebugEnabled()) {
-      LOG.debug(SchedulerEventType.MARK_CONTAINER_FOR_KILLABLE + ": container"
-          + killableContainer.toString());
-    }
-
-    if (!isLazyPreemptionEnabled) {
-      super.completedContainer(killableContainer, SchedulerUtils
-          .createPreemptedContainerStatus(killableContainer.getContainerId(),
-              SchedulerUtils.PREEMPTED_CONTAINER), RMContainerEventType.KILL);
-    } else {
-      FiCaSchedulerNode node = (FiCaSchedulerNode) getSchedulerNode(
-          killableContainer.getAllocatedNode());
-
-      FiCaSchedulerApp application = getCurrentAttemptForContainer(
-          killableContainer.getContainerId());
-
-      node.markContainerToKillable(killableContainer.getContainerId());
-
-      // notify PreemptionManager
-      // Get the application for the finished container
-      if (null != application) {
-        String leafQueueName = application.getCSLeafQueue().getQueueName();
-        getPreemptionManager().addKillableContainer(
-            new KillableContainer(killableContainer, node.getPartition(),
-                leafQueueName));
-      }    }
-  }
-
-  private synchronized void markContainerForNonKillable(
-      RMContainer nonKillableContainer) {
+  @Override
+  public void killPreemptedContainer(RMContainer cont) {
     if (LOG.isDebugEnabled()) {
-      LOG.debug(
-          SchedulerEventType.MARK_CONTAINER_FOR_NONKILLABLE + ": container"
-              + nonKillableContainer.toString());
-    }
-
-    FiCaSchedulerNode node = (FiCaSchedulerNode) getSchedulerNode(
-        nonKillableContainer.getAllocatedNode());
-
-    FiCaSchedulerApp application = getCurrentAttemptForContainer(
-        nonKillableContainer.getContainerId());
-
-    node.markContainerToNonKillable(nonKillableContainer.getContainerId());
-
-    // notify PreemptionManager
-    // Get the application for the finished container
-    if (null != application) {
-      String leafQueueName = application.getCSLeafQueue().getQueueName();
-      getPreemptionManager().removeKillableContainer(
-          new KillableContainer(nonKillableContainer, node.getPartition(),
-              leafQueueName));
+      LOG.debug(SchedulerEventType.KILL_PREEMPTED_CONTAINER + ": container"
+          + cont.toString());
     }
+    super.completedContainer(cont, SchedulerUtils
+        .createPreemptedContainerStatus(cont.getContainerId(),
+        SchedulerUtils.PREEMPTED_CONTAINER), RMContainerEventType.KILL);
   }
 
   @Override
@@ -2004,7 +1945,6 @@ public class CapacityScheduler extends
     return ret;
   }
 
-  @Override
   public SchedulerHealth getSchedulerHealth() {
     return this.schedulerHealth;
   }
@@ -2014,11 +1954,6 @@ public class CapacityScheduler extends
   }
 
   @Override
-  public long getLastNodeUpdateTime() {
-    return lastNodeUpdateTime;
-  }
-
-  @Override
   public Priority checkAndGetApplicationPriority(Priority priorityFromContext,
       String user, String queueName, ApplicationId applicationId)
       throws YarnException {
@@ -2119,9 +2054,4 @@ public class CapacityScheduler extends
         + rmApp.getQueue() + " for application: " + applicationId
         + " for the user: " + rmApp.getUser());
   }
-
-  @Override
-  public PreemptionManager getPreemptionManager() {
-    return preemptionManager;
-  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fa7a4352/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 3729264..3756d9e 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
@@ -257,12 +257,6 @@ public class CapacitySchedulerConfiguration extends ReservationSchedulerConfigur
   public static final String RESERVATION_ENFORCEMENT_WINDOW =
       "reservation-enforcement-window";
 
-  @Private
-  public static final String LAZY_PREEMPTION_ENALBED = PREFIX + "lazy-preemption-enabled";
-
-  @Private
-  public static final boolean DEFAULT_LAZY_PREEMPTION_ENABLED = false;
-
   public CapacitySchedulerConfiguration() {
     this(new Configuration());
   }
@@ -1013,11 +1007,7 @@ public class CapacitySchedulerConfiguration extends ReservationSchedulerConfigur
   @VisibleForTesting
   public void setOrderingPolicyParameter(String queue,
       String parameterKey, String parameterValue) {
-    set(getQueuePrefix(queue) + ORDERING_POLICY + "." + parameterKey,
-        parameterValue);
-  }
-
-  public boolean getLazyPreemptionEnabled() {
-    return getBoolean(LAZY_PREEMPTION_ENALBED, DEFAULT_LAZY_PREEMPTION_ENABLED);
+    set(getQueuePrefix(queue) + ORDERING_POLICY + "."
+        + parameterKey, parameterValue);
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fa7a4352/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 1203272..2a0dd0d 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
@@ -18,20 +18,17 @@
 
 package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity;
 
+import java.util.Comparator;
+
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerHealth;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.preemption.PreemptionManager;
 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.server.resourcemanager.security.RMContainerTokenSecretManager;
 import org.apache.hadoop.yarn.util.resource.ResourceCalculator;
 
-import java.util.Comparator;
-
 /**
  * Read-only interface to {@link CapacityScheduler} context.
  */
@@ -64,12 +61,4 @@ public interface CapacitySchedulerContext {
   PartitionedQueueComparator getPartitionedQueueComparator();
   
   FiCaSchedulerNode getNode(NodeId nodeId);
-
-  FiCaSchedulerApp getApplicationAttempt(ApplicationAttemptId attemptId);
-
-  PreemptionManager getPreemptionManager();
-
-  SchedulerHealth getSchedulerHealth();
-
-  long getLastNodeUpdateTime();
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fa7a4352/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 3dc2090..c625fae 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
@@ -37,11 +37,9 @@ import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.security.AccessControlException;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.authorize.AccessControlList;
-import org.apache.hadoop.util.Time;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.Container;
-import org.apache.hadoop.yarn.api.records.ContainerExitStatus;
 import org.apache.hadoop.yarn.api.records.ContainerStatus;
 import org.apache.hadoop.yarn.api.records.Priority;
 import org.apache.hadoop.yarn.api.records.QueueACL;
@@ -65,9 +63,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceUsage;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedContainerChangeRequest;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplicationAttempt;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplicationAttempt.AMState;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerHealth;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerUtils;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.preemption.KillableContainer;
 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.server.resourcemanager.scheduler.policy.FifoOrderingPolicyForPendingApps;
@@ -827,40 +823,6 @@ public class LeafQueue extends AbstractCSQueue {
       assignment.setExcessReservation(null);
     }
   }
-
-  private void killToPreemptContainers(Resource clusterResource,
-      FiCaSchedulerNode node,
-      CSAssignment assignment) {
-    if (assignment.getContainersToKill() != null) {
-      StringBuilder sb = new StringBuilder("Killing containers: [");
-
-      for (RMContainer c : assignment.getContainersToKill()) {
-        FiCaSchedulerApp application = csContext.getApplicationAttempt(
-            c.getApplicationAttemptId());
-        LeafQueue q = application.getCSLeafQueue();
-        q.completedContainer(clusterResource, application, node, c, SchedulerUtils
-                .createPreemptedContainerStatus(c.getContainerId(),
-                    SchedulerUtils.PREEMPTED_CONTAINER), RMContainerEventType.KILL,
-            null, false);
-        sb.append("(container=" + c.getContainerId() + " resource=" + c
-            .getAllocatedResource() + ")");
-      }
-
-      sb.append("] for container=" + assignment.getAssignmentInformation()
-          .getFirstAllocatedOrReservedContainerId() + " resource=" + assignment
-          .getResource());
-      LOG.info(sb.toString());
-
-    }
-  }
-
-  private void setPreemptionAllowed(ResourceLimits limits, String nodePartition) {
-    // Set preemption-allowed:
-    // For leaf queue, only under-utilized queue is allowed to preempt resources from other queues
-    float usedCapacity = queueCapacities.getAbsoluteUsedCapacity(nodePartition);
-    float guaranteedCapacity = queueCapacities.getAbsoluteCapacity(nodePartition);
-    limits.setIsAllowPreemption(usedCapacity < guaranteedCapacity);
-  }
   
   @Override
   public synchronized CSAssignment assignContainers(Resource clusterResource,
@@ -873,8 +835,6 @@ public class LeafQueue extends AbstractCSQueue {
           + " #applications=" + orderingPolicy.getNumSchedulableEntities());
     }
 
-    setPreemptionAllowed(currentResourceLimits, node.getPartition());
-
     // Check for reserved resources
     RMContainer reservedContainer = node.getReservedContainer();
     if (reservedContainer != null) {
@@ -886,7 +846,6 @@ public class LeafQueue extends AbstractCSQueue {
                 currentResourceLimits, schedulingMode, reservedContainer);
         handleExcessReservedContainer(clusterResource, assignment, node,
             application);
-        killToPreemptContainers(clusterResource, node, assignment);
         return assignment;
       }
     }
@@ -948,7 +907,6 @@ public class LeafQueue extends AbstractCSQueue {
       
       handleExcessReservedContainer(clusterResource, assignment, node,
           application);
-      killToPreemptContainers(clusterResource, node, assignment);
 
       if (Resources.greaterThan(resourceCalculator, clusterResource, assigned,
           Resources.none())) {
@@ -1252,34 +1210,11 @@ public class LeafQueue extends AbstractCSQueue {
     }
   }
 
-  private void updateSchedulerHealthForCompletedContainer(
-      RMContainer rmContainer, ContainerStatus containerStatus) {
-    // Update SchedulerHealth for released / preempted container
-    SchedulerHealth schedulerHealth = csContext.getSchedulerHealth();
-    if (null == schedulerHealth) {
-      // Only do update if we have schedulerHealth
-      return;
-    }
-
-    if (containerStatus.getExitStatus() == ContainerExitStatus.PREEMPTED) {
-      schedulerHealth.updatePreemption(Time.now(), rmContainer.getAllocatedNode(),
-          rmContainer.getContainerId(), getQueuePath());
-      schedulerHealth.updateSchedulerPreemptionCounts(1);
-    } else {
-      schedulerHealth.updateRelease(csContext.getLastNodeUpdateTime(),
-          rmContainer.getAllocatedNode(), rmContainer.getContainerId(),
-          getQueuePath());
-    }
-  }
-
   @Override
   public void completedContainer(Resource clusterResource, 
       FiCaSchedulerApp application, FiCaSchedulerNode node, RMContainer rmContainer, 
       ContainerStatus containerStatus, RMContainerEventType event, CSQueue childQueue,
       boolean sortQueues) {
-    // Update SchedulerHealth for released / preempted container
-    updateSchedulerHealthForCompletedContainer(rmContainer, containerStatus);
-
     if (application != null) {
       // unreserve container increase request if it previously reserved.
       if (rmContainer.hasIncreaseReservation()) {
@@ -1330,10 +1265,6 @@ public class LeafQueue extends AbstractCSQueue {
           rmContainer, null, event, this, sortQueues);
       }
     }
-
-    // Notify PreemptionManager
-    csContext.getPreemptionManager().removeKillableContainer(
-        new KillableContainer(rmContainer, node.getPartition(), queueName));
   }
 
   synchronized void allocateResource(Resource clusterResource,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fa7a4352/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 6fcd6c1..7cf5565 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
@@ -18,6 +18,18 @@
 
 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;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.TreeSet;
+
 import org.apache.commons.lang.StringUtils;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -37,7 +49,6 @@ import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.exceptions.InvalidResourceRequestException;
 import org.apache.hadoop.yarn.factories.RecordFactory;
 import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
-import org.apache.hadoop.yarn.nodelabels.RMNodeLabel;
 import org.apache.hadoop.yarn.security.AccessType;
 import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager;
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
@@ -46,25 +57,12 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerStat
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ActiveUsersManager;
 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.SchedContainerChangeRequest;
 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.SchedContainerChangeRequest;
 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;
 
-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;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.TreeSet;
-
 @Private
 @Evolving
 public class ParentQueue extends AbstractCSQueue {
@@ -388,11 +386,6 @@ public class ParentQueue extends AbstractCSQueue {
     // if our queue cannot access this node, just return
     if (schedulingMode == SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY
         && !accessibleToPartition(node.getPartition())) {
-      if (LOG.isDebugEnabled()) {
-        LOG.debug("Skip this queue=" + getQueuePath()
-            + ", because it is not able to access partition=" + node
-            .getPartition());
-      }
       return CSAssignment.NULL_ASSIGNMENT;
     }
     
@@ -438,7 +431,7 @@ public class ParentQueue extends AbstractCSQueue {
               resourceCalculator, clusterResource, 
               assignedToChild.getResource(), Resources.none())) {
         // Track resource utilization for the parent-queue
-        allocateResource(clusterResource, assignedToChild.getResource(),
+        super.allocateResource(clusterResource, assignedToChild.getResource(),
             node.getPartition(), assignedToChild.isIncreasedAllocation());
         
         // Track resource utilization in this pass of the scheduler
@@ -501,38 +494,29 @@ public class ParentQueue extends AbstractCSQueue {
   }
 
   private boolean canAssign(Resource clusterResource, FiCaSchedulerNode node) {
-    // Two conditions need to meet when trying to allocate:
-    // 1) Node doesn't have reserved container
-    // 2) Node's available-resource + killable-resource should > 0
-    return node.getReservedContainer() == null && Resources.greaterThanOrEqual(
-        resourceCalculator, clusterResource, Resources
-            .add(node.getUnallocatedResource(), node.getTotalKillableResources()),
-        minimumAllocation);
+    return (node.getReservedContainer() == null) && 
+        Resources.greaterThanOrEqual(resourceCalculator, clusterResource, 
+            node.getUnallocatedResource(), minimumAllocation);
   }
-
+  
   private ResourceLimits getResourceLimitsOfChild(CSQueue child,
-      Resource clusterResource, ResourceLimits parentLimits,
-      String nodePartition) {
+      Resource clusterResource, ResourceLimits parentLimits) {
     // Set resource-limit of a given child, child.limit =
     // min(my.limit - my.used + child.used, child.max)
 
     // Parent available resource = parent-limit - parent-used-resource
-    Resource parentMaxAvailableResource = Resources.subtract(
-        parentLimits.getLimit(), queueUsage.getUsed(nodePartition));
-    // Deduct killable from used
-    Resources.addTo(parentMaxAvailableResource,
-        getTotalKillableResource(nodePartition));
+    Resource parentMaxAvailableResource =
+        Resources.subtract(parentLimits.getLimit(), getUsedResources());
 
     // Child's limit = parent-available-resource + child-used
-    Resource childLimit = Resources.add(parentMaxAvailableResource,
-        child.getQueueResourceUsage().getUsed(nodePartition));
+    Resource childLimit =
+        Resources.add(parentMaxAvailableResource, child.getUsedResources());
 
     // Get child's max resource
-    Resource childConfiguredMaxResource = Resources.multiplyAndNormalizeDown(
-        resourceCalculator,
-        labelManager.getResourceByLabel(nodePartition, clusterResource),
-        child.getQueueCapacities().getAbsoluteMaximumCapacity(nodePartition),
-        minimumAllocation);
+    Resource childConfiguredMaxResource =
+        Resources.multiplyAndNormalizeDown(resourceCalculator, labelManager
+            .getResourceByLabel(RMNodeLabelsManager.NO_LABEL, clusterResource),
+            child.getAbsoluteMaximumCapacity(), minimumAllocation);
 
     // Child's limit should be capped by child configured max resource
     childLimit =
@@ -584,7 +568,7 @@ public class ParentQueue extends AbstractCSQueue {
 
       // Get ResourceLimits of child queue before assign containers
       ResourceLimits childLimits =
-          getResourceLimitsOfChild(childQueue, cluster, limits, node.getPartition());
+          getResourceLimitsOfChild(childQueue, cluster, limits);
       
       assignment = childQueue.assignContainers(cluster, node, 
           childLimits, schedulingMode);
@@ -730,8 +714,8 @@ public class ParentQueue extends AbstractCSQueue {
     // Update all children
     for (CSQueue childQueue : childQueues) {
       // Get ResourceLimits of child queue before assign containers
-      ResourceLimits childLimits = getResourceLimitsOfChild(childQueue,
-          clusterResource, resourceLimits, RMNodeLabelsManager.NO_LABEL);
+      ResourceLimits childLimits =
+          getResourceLimitsOfChild(childQueue, clusterResource, resourceLimits);     
       childQueue.updateClusterResource(clusterResource, childLimits);
     }
     
@@ -754,8 +738,8 @@ public class ParentQueue extends AbstractCSQueue {
     synchronized (this) {
       FiCaSchedulerNode node =
           scheduler.getNode(rmContainer.getContainer().getNodeId());
-      allocateResource(clusterResource,
-          rmContainer.getContainer().getResource(), node.getPartition(), false);
+      super.allocateResource(clusterResource, rmContainer.getContainer()
+          .getResource(), node.getPartition(), false);
     }
     if (parent != null) {
       parent.recoverContainer(clusterResource, attempt, rmContainer);
@@ -782,7 +766,7 @@ public class ParentQueue extends AbstractCSQueue {
     if (application != null) {
       FiCaSchedulerNode node =
           scheduler.getNode(rmContainer.getContainer().getNodeId());
-      allocateResource(clusterResource, rmContainer.getContainer()
+      super.allocateResource(clusterResource, rmContainer.getContainer()
           .getResource(), node.getPartition(), false);
       LOG.info("movedContainer" + " queueMoveIn=" + getQueueName()
           + " usedCapacity=" + getUsedCapacity() + " absoluteUsedCapacity="
@@ -818,79 +802,4 @@ public class ParentQueue extends AbstractCSQueue {
   public synchronized int getNumApplications() {
     return numApplications;
   }
-
-  synchronized void allocateResource(Resource clusterResource,
-      Resource resource, String nodePartition, boolean changeContainerResource) {
-    super.allocateResource(clusterResource, resource, nodePartition,
-        changeContainerResource);
-
-    /**
-     * check if we need to kill (killable) containers if maximum resource violated.
-     * Doing this because we will deduct killable resource when going from root.
-     * For example:
-     * <pre>
-     *      Root
-     *      /   \
-     *     a     b
-     *   /  \
-     *  a1  a2
-     * </pre>
-     *
-     * a: max=10G, used=10G, killable=2G
-     * a1: used=8G, killable=2G
-     * a2: used=2G, pending=2G, killable=0G
-     *
-     * When we get queue-a to allocate resource, even if queue-a
-     * reaches its max resource, we deduct its used by killable, so we can allocate
-     * at most 2G resources. ResourceLimits passed down to a2 has headroom set to 2G.
-     *
-     * If scheduler finds a 2G available resource in existing cluster, and assigns it
-     * to a2, now a2's used= 2G + 2G = 4G, and a's used = 8G + 4G = 12G > 10G
-     *
-     * When this happens, we have to preempt killable container (on same or different
-     * nodes) of parent queue to avoid violating parent's max resource.
-     */
-    if (getQueueCapacities().getAbsoluteMaximumCapacity(nodePartition)
-        < getQueueCapacities().getAbsoluteUsedCapacity(nodePartition)) {
-      killContainersToEnforceMaxQueueCapacity(nodePartition, clusterResource);
-    }
-  }
-
-  private void killContainersToEnforceMaxQueueCapacity(String partition,
-      Resource clusterResource) {
-    Iterator<RMContainer> killableContainerIter = getKillableContainers(
-        partition);
-    if (!killableContainerIter.hasNext()) {
-      return;
-    }
-
-    Resource partitionResource = labelManager.getResourceByLabel(partition,
-        null);
-    Resource maxResource = Resources.multiply(partitionResource,
-        getQueueCapacities().getAbsoluteMaximumCapacity(partition));
-
-    while (Resources.greaterThan(resourceCalculator, partitionResource,
-        queueUsage.getUsed(partition), maxResource)) {
-      RMContainer toKillContainer = killableContainerIter.next();
-      FiCaSchedulerApp attempt = csContext.getApplicationAttempt(
-          toKillContainer.getContainerId().getApplicationAttemptId());
-      FiCaSchedulerNode node = csContext.getNode(
-          toKillContainer.getAllocatedNode());
-      if (null != attempt && null != node) {
-        LeafQueue lq = attempt.getCSLeafQueue();
-        lq.completedContainer(clusterResource, attempt, node, toKillContainer,
-            SchedulerUtils.createPreemptedContainerStatus(
-                toKillContainer.getContainerId(),
-                SchedulerUtils.PREEMPTED_CONTAINER), RMContainerEventType.KILL,
-            null, false);
-        LOG.info("Killed container=" + toKillContainer.getContainerId()
-            + " from queue=" + lq.getQueueName() + " to make queue=" + this
-            .getQueueName() + "'s max-capacity enforced");
-      }
-
-      if (!killableContainerIter.hasNext()) {
-        break;
-      }
-    }
-  }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fa7a4352/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/allocator/AbstractContainerAllocator.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/allocator/AbstractContainerAllocator.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/allocator/AbstractContainerAllocator.java
index afac235..ee01bd1 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/allocator/AbstractContainerAllocator.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/allocator/AbstractContainerAllocator.java
@@ -108,8 +108,6 @@ public abstract class AbstractContainerAllocator {
           assignment.setFulfilledReservation(true);
         }
       }
-
-      assignment.setContainersToKill(result.getToKillContainers());
     }
     
     return assignment;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fa7a4352/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/allocator/ContainerAllocation.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/allocator/ContainerAllocation.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/allocator/ContainerAllocation.java
index 8f749f6..1df9410 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/allocator/ContainerAllocation.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/allocator/ContainerAllocation.java
@@ -19,14 +19,11 @@
 package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.allocator;
 
 import org.apache.hadoop.yarn.api.records.Container;
-import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.NodeType;
 import org.apache.hadoop.yarn.util.resource.Resources;
 
-import java.util.List;
-
 public class ContainerAllocation {
   /**
    * Skip the locality (e.g. node-local, rack-local, any), and look at other
@@ -59,7 +56,6 @@ public class ContainerAllocation {
   NodeType containerNodeType = NodeType.NODE_LOCAL;
   NodeType requestNodeType = NodeType.NODE_LOCAL;
   Container updatedContainer;
-  private List<RMContainer> toKillContainers;
 
   public ContainerAllocation(RMContainer containerToBeUnreserved,
       Resource resourceToBeAllocated, AllocationState state) {
@@ -90,12 +86,4 @@ public class ContainerAllocation {
   public Container getUpdatedContainer() {
     return updatedContainer;
   }
-
-  public void setToKillContainers(List<RMContainer> toKillContainers) {
-    this.toKillContainers = toKillContainers;
-  }
-
-  public List<RMContainer> getToKillContainers() {
-    return toKillContainers;
-  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fa7a4352/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/allocator/RegularContainerAllocator.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/allocator/RegularContainerAllocator.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/allocator/RegularContainerAllocator.java
index a5ca2d8..e168edf 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/allocator/RegularContainerAllocator.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/allocator/RegularContainerAllocator.java
@@ -42,9 +42,6 @@ import org.apache.hadoop.yarn.server.utils.BuilderUtils;
 import org.apache.hadoop.yarn.util.resource.ResourceCalculator;
 import org.apache.hadoop.yarn.util.resource.Resources;
 
-import java.util.ArrayList;
-import java.util.List;
-
 /**
  * Allocate normal (new) containers, considers locality/label, etc. Using
  * delayed scheduling mechanism to get better locality allocation.
@@ -438,6 +435,9 @@ public class RegularContainerAllocator extends AbstractContainerAllocator {
       return ContainerAllocation.LOCALITY_SKIPPED;
     }
 
+    assert Resources.greaterThan(
+        rc, clusterResource, available, Resources.none());
+
     boolean shouldAllocOrReserveNewContainer = shouldAllocOrReserveNewContainer(
         priority, capability);
 
@@ -460,29 +460,6 @@ public class RegularContainerAllocator extends AbstractContainerAllocator {
     boolean reservationsContinueLooking =
         application.getCSLeafQueue().getReservationContinueLooking();
 
-    // Check if we need to kill some containers to allocate this one
-    List<RMContainer> toKillContainers = null;
-    if (availableContainers == 0 && currentResoureLimits.isAllowPreemption()) {
-      Resource availableAndKillable = Resources.clone(available);
-      for (RMContainer killableContainer : node
-          .getKillableContainers().values()) {
-        if (null == toKillContainers) {
-          toKillContainers = new ArrayList<>();
-        }
-        toKillContainers.add(killableContainer);
-        Resources.addTo(availableAndKillable,
-                        killableContainer.getAllocatedResource());
-        if (Resources.fitsIn(rc,
-                             clusterResource,
-                             capability,
-                             availableAndKillable)) {
-          // Stop if we find enough spaces
-          availableContainers = 1;
-          break;
-        }
-      }
-    }
-
     if (availableContainers > 0) {
       // Allocate...
       // We will only do continuous reservation when this is not allocated from
@@ -522,12 +499,12 @@ public class RegularContainerAllocator extends AbstractContainerAllocator {
           new ContainerAllocation(unreservedContainer, request.getCapability(),
               AllocationState.ALLOCATED);
       result.containerNodeType = type;
-      result.setToKillContainers(toKillContainers);
       return result;
     } else {
       // if we are allowed to allocate but this node doesn't have space, reserve
       // it or if this was an already a reserved container, reserve it again
       if (shouldAllocOrReserveNewContainer || rmContainer != null) {
+
         if (reservationsContinueLooking && rmContainer == null) {
           // we could possibly ignoring queue capacity or user limits when
           // reservationsContinueLooking is set. Make sure we didn't need to
@@ -545,7 +522,6 @@ public class RegularContainerAllocator extends AbstractContainerAllocator {
             new ContainerAllocation(null, request.getCapability(),
                 AllocationState.RESERVED);
         result.containerNodeType = type;
-        result.setToKillContainers(null);
         return result;
       }
       // Skip the locality request
@@ -637,7 +613,8 @@ public class RegularContainerAllocator extends AbstractContainerAllocator {
   }
 
   ContainerAllocation doAllocation(ContainerAllocation allocationResult,
-      FiCaSchedulerNode node, Priority priority,
+      Resource clusterResource, FiCaSchedulerNode node,
+      SchedulingMode schedulingMode, Priority priority,
       RMContainer reservedContainer) {
     // Create the container if necessary
     Container container =
@@ -701,7 +678,9 @@ public class RegularContainerAllocator extends AbstractContainerAllocator {
 
     if (AllocationState.ALLOCATED == result.state
         || AllocationState.RESERVED == result.state) {
-      result = doAllocation(result, node, priority, reservedContainer);
+      result =
+          doAllocation(result, clusterResource, node, schedulingMode, priority,
+              reservedContainer);
     }
 
     return result;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fa7a4352/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/preemption/KillableContainer.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/preemption/KillableContainer.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/preemption/KillableContainer.java
deleted file mode 100644
index 675b0b4..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/preemption/KillableContainer.java
+++ /dev/null
@@ -1,45 +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
- * <p/>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p/>
- * 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.preemption;
-
-import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
-
-public class KillableContainer {
-  RMContainer container;
-  String partition;
-  String leafQueueName;
-
-  public KillableContainer(RMContainer container, String partition, String leafQueueName) {
-    this.container = container;
-    this.partition = partition;
-    this.leafQueueName = leafQueueName;
-  }
-
-  public RMContainer getRMContainer() {
-    return this.container;
-  }
-
-  public String getNodePartition() {
-    return this.partition;
-  }
-
-  public String getLeafQueueName() {
-    return this.leafQueueName;
-  }
-}


[33/46] hadoop git commit: Revert "CapacityScheduler: Improve preemption to only kill containers that would satisfy the incoming request. (Wangda Tan)"

Posted by ae...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/fa7a4352/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/preemption/PreemptableQueue.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/preemption/PreemptableQueue.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/preemption/PreemptableQueue.java
deleted file mode 100644
index 19148d7..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/preemption/PreemptableQueue.java
+++ /dev/null
@@ -1,102 +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
- * <p/>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p/>
- * 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.preemption;
-
-import org.apache.hadoop.yarn.api.records.ContainerId;
-import org.apache.hadoop.yarn.api.records.Resource;
-import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
-import org.apache.hadoop.yarn.util.resource.Resources;
-
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.Map;
-import java.util.concurrent.ConcurrentSkipListMap;
-
-public class PreemptableQueue {
-  // Partition -> killable resources and containers
-  private Map<String, Resource> totalKillableResources = new HashMap<>();
-  private Map<String, Map<ContainerId, RMContainer>> killableContainers =
-      new HashMap<>();
-  private PreemptableQueue parent;
-
-  public PreemptableQueue(PreemptableQueue parent) {
-    this.parent = parent;
-  }
-
-  public PreemptableQueue(Map<String, Resource> totalKillableResources,
-      Map<String, Map<ContainerId, RMContainer>> killableContainers) {
-    this.totalKillableResources = totalKillableResources;
-    this.killableContainers = killableContainers;
-  }
-
-  void addKillableContainer(KillableContainer container) {
-    String partition = container.getNodePartition();
-    if (!totalKillableResources.containsKey(partition)) {
-      totalKillableResources.put(partition, Resources.createResource(0));
-      killableContainers.put(partition,
-          new ConcurrentSkipListMap<ContainerId, RMContainer>());
-    }
-
-    RMContainer c = container.getRMContainer();
-    Resources.addTo(totalKillableResources.get(partition),
-        c.getAllocatedResource());
-    killableContainers.get(partition).put(c.getContainerId(), c);
-
-    if (null != parent) {
-      parent.addKillableContainer(container);
-    }
-  }
-
-  void removeKillableContainer(KillableContainer container) {
-    String partition = container.getNodePartition();
-    Map<ContainerId, RMContainer> partitionKillableContainers =
-        killableContainers.get(partition);
-    if (partitionKillableContainers != null) {
-      RMContainer rmContainer = partitionKillableContainers.remove(
-          container.getRMContainer().getContainerId());
-      if (null != rmContainer) {
-        Resources.subtractFrom(totalKillableResources.get(partition),
-            rmContainer.getAllocatedResource());
-      }
-    }
-
-    if (null != parent) {
-      parent.removeKillableContainer(container);
-    }
-  }
-
-  public Resource getKillableResource(String partition) {
-    Resource res = totalKillableResources.get(partition);
-    return res == null ? Resources.none() : res;
-  }
-
-  @SuppressWarnings("unchecked")
-  public Map<ContainerId, RMContainer> getKillableContainers(String partition) {
-    Map<ContainerId, RMContainer> map = killableContainers.get(partition);
-    return map == null ? Collections.EMPTY_MAP : map;
-  }
-
-  public Map<String, Map<ContainerId, RMContainer>> getKillableContainers() {
-    return killableContainers;
-  }
-
-  Map<String, Resource> getTotalKillableResources() {
-    return totalKillableResources;
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fa7a4352/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/preemption/PreemptionManager.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/preemption/PreemptionManager.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/preemption/PreemptionManager.java
deleted file mode 100644
index a9f02a5..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/preemption/PreemptionManager.java
+++ /dev/null
@@ -1,165 +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
- * <p/>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p/>
- * 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.preemption;
-
-import com.google.common.annotations.VisibleForTesting;
-import org.apache.hadoop.yarn.api.records.ContainerId;
-import org.apache.hadoop.yarn.api.records.Resource;
-import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CSQueue;
-import org.apache.hadoop.yarn.util.resource.Resources;
-
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.Map;
-import java.util.concurrent.locks.ReentrantReadWriteLock;
-
-public class PreemptionManager {
-  private ReentrantReadWriteLock.ReadLock readLock;
-  private ReentrantReadWriteLock.WriteLock writeLock;
-  private Map<String, PreemptableQueue> entities = new HashMap<>();
-
-  public PreemptionManager() {
-    ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
-    readLock = lock.readLock();
-    writeLock = lock.writeLock();
-  }
-
-  public void refreshQueues(CSQueue parent, CSQueue current) {
-    try {
-      writeLock.lock();
-      PreemptableQueue parentEntity = null;
-      if (parent != null) {
-        parentEntity = entities.get(parent.getQueueName());
-      }
-
-      if (!entities.containsKey(current.getQueueName())) {
-        entities.put(current.getQueueName(),
-            new PreemptableQueue(parentEntity));
-      }
-
-      if (current.getChildQueues() != null) {
-        for (CSQueue child : current.getChildQueues()) {
-          refreshQueues(current, child);
-        }
-      }
-    }
-    finally {
-      writeLock.unlock();
-    }
-  }
-
-  public void addKillableContainer(KillableContainer container) {
-    try {
-      writeLock.lock();
-      PreemptableQueue entity = entities.get(container.getLeafQueueName());
-      if (null != entity) {
-        entity.addKillableContainer(container);
-      }
-    }
-    finally {
-      writeLock.unlock();
-    }
-  }
-
-  public void removeKillableContainer(KillableContainer container) {
-    try {
-      writeLock.lock();
-      PreemptableQueue entity = entities.get(container.getLeafQueueName());
-      if (null != entity) {
-        entity.removeKillableContainer(container);
-      }
-    }
-    finally {
-      writeLock.unlock();
-    }
-  }
-
-  public void moveKillableContainer(KillableContainer oldContainer,
-      KillableContainer newContainer) {
-    // TODO, will be called when partition of the node changed OR
-    // container moved to different queue
-  }
-
-  public void updateKillableContainerResource(KillableContainer container,
-      Resource oldResource, Resource newResource) {
-    // TODO, will be called when container's resource changed
-  }
-
-  @VisibleForTesting
-  public Map<ContainerId, RMContainer> getKillableContainersMap(
-      String queueName, String partition) {
-    try {
-      readLock.lock();
-      PreemptableQueue entity = entities.get(queueName);
-      if (entity != null) {
-        Map<ContainerId, RMContainer> containers =
-            entity.getKillableContainers().get(partition);
-        if (containers != null) {
-          return containers;
-        }
-      }
-      return Collections.emptyMap();
-    }
-    finally {
-      readLock.unlock();
-    }
-  }
-
-  public Iterator<RMContainer> getKillableContainers(String queueName,
-      String partition) {
-    return getKillableContainersMap(queueName, partition).values().iterator();
-  }
-
-  public Resource getKillableResource(String queueName, String partition) {
-    try {
-      readLock.lock();
-      PreemptableQueue entity = entities.get(queueName);
-      if (entity != null) {
-        Resource res = entity.getTotalKillableResources().get(partition);
-        if (res == null || res.equals(Resources.none())) {
-          return Resources.none();
-        }
-        return Resources.clone(res);
-      }
-      return Resources.none();
-    }
-    finally {
-      readLock.unlock();
-    }
-  }
-
-  public Map<String, PreemptableQueue> getShallowCopyOfPreemptableEntities() {
-    try {
-      readLock.lock();
-      Map<String, PreemptableQueue> map = new HashMap<>();
-      for (Map.Entry<String, PreemptableQueue> entry : entities.entrySet()) {
-        String key = entry.getKey();
-        PreemptableQueue entity = entry.getValue();
-        map.put(key, new PreemptableQueue(
-            new HashMap<>(entity.getTotalKillableResources()),
-            new HashMap<>(entity.getKillableContainers())));
-      }
-      return map;
-    } finally {
-      readLock.unlock();
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fa7a4352/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/common/AssignmentInformation.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/AssignmentInformation.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/AssignmentInformation.java
index aad3bc7..5158255 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/AssignmentInformation.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/AssignmentInformation.java
@@ -120,9 +120,9 @@ public class AssignmentInformation {
   }
 
   private ContainerId getFirstContainerIdFromOperation(Operation op) {
-    if (null != operationDetails.get(op)) {
+    if (null != operationDetails.get(Operation.ALLOCATION)) {
       List<AssignmentDetails> assignDetails =
-          operationDetails.get(op);
+          operationDetails.get(Operation.ALLOCATION);
       if (!assignDetails.isEmpty()) {
         return assignDetails.get(0).containerId;
       }
@@ -131,7 +131,7 @@ public class AssignmentInformation {
   }
 
   public ContainerId getFirstAllocatedOrReservedContainerId() {
-    ContainerId containerId;
+    ContainerId containerId = null;
     containerId = getFirstContainerIdFromOperation(Operation.ALLOCATION);
     if (null != containerId) {
       return containerId;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fa7a4352/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 f474aad..4d563cd 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
@@ -59,7 +59,6 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CSAMCont
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CSAssignment;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityHeadroomProvider;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerContext;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.LeafQueue;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueueCapacities;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.SchedulingMode;
@@ -95,7 +94,6 @@ public class FiCaSchedulerApp extends SchedulerApplicationAttempt {
    * to hold the message if its app doesn't not get container from a node
    */
   private String appSkipNodeDiagnostics;
-  private CapacitySchedulerContext capacitySchedulerContext;
 
   public FiCaSchedulerApp(ApplicationAttemptId applicationAttemptId, 
       String user, Queue queue, ActiveUsersManager activeUsersManager,
@@ -140,30 +138,28 @@ public class FiCaSchedulerApp extends SchedulerApplicationAttempt {
     }
     
     containerAllocator = new ContainerAllocator(this, rc, rmContext);
-
-    if (scheduler instanceof CapacityScheduler) {
-      capacitySchedulerContext = (CapacitySchedulerContext) scheduler;
-    }
   }
 
-  public synchronized boolean containerCompleted(RMContainer rmContainer,
+  synchronized public boolean containerCompleted(RMContainer rmContainer,
       ContainerStatus containerStatus, RMContainerEventType event,
       String partition) {
-    ContainerId containerId = rmContainer.getContainerId();
 
     // Remove from the list of containers
-    if (null == liveContainers.remove(containerId)) {
+    if (null == liveContainers.remove(rmContainer.getContainerId())) {
       return false;
     }
-
+    
     // Remove from the list of newly allocated containers if found
     newlyAllocatedContainers.remove(rmContainer);
 
+    Container container = rmContainer.getContainer();
+    ContainerId containerId = container.getId();
+
     // Inform the container
     rmContainer.handle(
         new RMContainerFinishedEvent(containerId, containerStatus, event));
 
-    containersToPreempt.remove(containerId);
+    containersToPreempt.remove(rmContainer.getContainerId());
 
     RMAuditLogger.logSuccess(getUser(),
         AuditConstants.RELEASE_CONTAINER, "SchedulerApp",
@@ -180,7 +176,7 @@ public class FiCaSchedulerApp extends SchedulerApplicationAttempt {
     return true;
   }
 
-  public synchronized RMContainer allocate(NodeType type, FiCaSchedulerNode node,
+  synchronized public RMContainer allocate(NodeType type, FiCaSchedulerNode node,
       Priority priority, ResourceRequest request, 
       Container container) {
 
@@ -204,9 +200,7 @@ public class FiCaSchedulerApp extends SchedulerApplicationAttempt {
 
     // Add it to allContainers list.
     newlyAllocatedContainers.add(rmContainer);
-
-    ContainerId containerId = container.getId();
-    liveContainers.put(containerId, rmContainer);
+    liveContainers.put(container.getId(), rmContainer);    
 
     // Update consumption and track allocations
     List<ResourceRequest> resourceRequestList = appSchedulingInfo.allocate(
@@ -219,17 +213,17 @@ public class FiCaSchedulerApp extends SchedulerApplicationAttempt {
 
     // Inform the container
     rmContainer.handle(
-        new RMContainerEvent(containerId, RMContainerEventType.START));
+        new RMContainerEvent(container.getId(), RMContainerEventType.START));
 
     if (LOG.isDebugEnabled()) {
       LOG.debug("allocate: applicationAttemptId=" 
-          + containerId.getApplicationAttemptId()
-          + " container=" + containerId + " host="
+          + container.getId().getApplicationAttemptId() 
+          + " container=" + container.getId() + " host="
           + container.getNodeId().getHost() + " type=" + type);
     }
     RMAuditLogger.logSuccess(getUser(),
         AuditConstants.ALLOC_CONTAINER, "SchedulerApp",
-        getApplicationId(), containerId);
+        getApplicationId(), container.getId());
     
     return rmContainer;
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fa7a4352/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/common/fica/FiCaSchedulerNode.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/FiCaSchedulerNode.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/FiCaSchedulerNode.java
index 1d0e78a..fe6db47 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/FiCaSchedulerNode.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/FiCaSchedulerNode.java
@@ -18,29 +18,22 @@
 
 package org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica;
 
+
+import java.util.Set;
+
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
-import org.apache.hadoop.yarn.api.records.Container;
-import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.Priority;
-import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.nodelabels.CommonNodeLabelsManager;
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplicationAttempt;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNode;
-import org.apache.hadoop.yarn.util.resource.Resources;
-
-import java.util.HashMap;
-import java.util.Map;
-import java.util.Set;
 
 public class FiCaSchedulerNode extends SchedulerNode {
 
   private static final Log LOG = LogFactory.getLog(FiCaSchedulerNode.class);
-  private Map<ContainerId, RMContainer> killableContainers = new HashMap<>();
-  private Resource totalKillableResources = Resource.newInstance(0, 0);
   
   public FiCaSchedulerNode(RMNode node, boolean usePortForNodeName,
       Set<String> nodeLabels) {
@@ -99,6 +92,7 @@ public class FiCaSchedulerNode extends SchedulerNode {
   @Override
   public synchronized void unreserveResource(
       SchedulerApplicationAttempt application) {
+
     // adding NP checks as this can now be called for preemption
     if (getReservedContainer() != null
         && getReservedContainer().getContainer() != null
@@ -121,55 +115,4 @@ public class FiCaSchedulerNode extends SchedulerNode {
     }
     setReservedContainer(null);
   }
-
-  // According to decisions from preemption policy, mark the container to killable
-  public synchronized void markContainerToKillable(ContainerId containerId) {
-    RMContainer c = launchedContainers.get(containerId);
-    if (c != null && !killableContainers.containsKey(containerId)) {
-      killableContainers.put(containerId, c);
-      Resources.addTo(totalKillableResources, c.getAllocatedResource());
-    }
-  }
-
-  // According to decisions from preemption policy, mark the container to
-  // non-killable
-  public synchronized void markContainerToNonKillable(ContainerId containerId) {
-    RMContainer c = launchedContainers.get(containerId);
-    if (c != null && killableContainers.containsKey(containerId)) {
-      killableContainers.remove(containerId);
-      Resources.subtractFrom(totalKillableResources, c.getAllocatedResource());
-    }
-  }
-
-  @Override
-  protected synchronized void updateResource(
-      Container container) {
-    super.updateResource(container);
-    if (killableContainers.containsKey(container.getId())) {
-      Resources.subtractFrom(totalKillableResources, container.getResource());
-      killableContainers.remove(container.getId());
-    }
-  }
-
-  @Override
-  protected synchronized void changeContainerResource(ContainerId containerId,
-      Resource deltaResource, boolean increase) {
-    super.changeContainerResource(containerId, deltaResource, increase);
-
-    if (killableContainers.containsKey(containerId)) {
-      if (increase) {
-        Resources.addTo(totalKillableResources, deltaResource);
-      } else {
-        Resources.subtractFrom(totalKillableResources, deltaResource);
-      }
-    }
-  }
-
-  public synchronized Resource getTotalKillableResources() {
-    return totalKillableResources;
-  }
-
-  public synchronized Map<ContainerId, RMContainer> getKillableContainers() {
-    return killableContainers;
-  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fa7a4352/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/event/SchedulerEventType.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/event/SchedulerEventType.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/event/SchedulerEventType.java
index 35b7c14..9cf09e9 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/event/SchedulerEventType.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/event/SchedulerEventType.java
@@ -38,15 +38,10 @@ public enum SchedulerEventType {
   // Source: ContainerAllocationExpirer
   CONTAINER_EXPIRED,
 
-  /* Source: SchedulingEditPolicy */
+  // Source: SchedulingEditPolicy
   KILL_RESERVED_CONTAINER,
-
-  // Mark a container for preemption
-  MARK_CONTAINER_FOR_PREEMPTION,
-
-  // Mark a for-preemption container killable
-  MARK_CONTAINER_FOR_KILLABLE,
-
-  // Cancel a killable container
-  MARK_CONTAINER_FOR_NONKILLABLE
+  MARK_CONTAINER_FOR_PREEMPTION, // Mark a container for preemption
+                                 // in the near future
+  KILL_PREEMPTED_CONTAINER // Kill a container previously marked for
+                           // preemption
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fa7a4352/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMDispatcher.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/TestRMDispatcher.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMDispatcher.java
index c944752..d9306dd 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMDispatcher.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMDispatcher.java
@@ -59,7 +59,7 @@ public class TestRMDispatcher {
       rmDispatcher.getEventHandler().handle(event1);
       ContainerPreemptEvent event2 =
           new ContainerPreemptEvent(appAttemptId, container,
-            SchedulerEventType.MARK_CONTAINER_FOR_KILLABLE);
+            SchedulerEventType.KILL_PREEMPTED_CONTAINER);
       rmDispatcher.getEventHandler().handle(event2);
       ContainerPreemptEvent event3 =
           new ContainerPreemptEvent(appAttemptId, container,
@@ -70,7 +70,7 @@ public class TestRMDispatcher {
       verify(sched, times(3)).handle(any(SchedulerEvent.class));
       verify(sched).killReservedContainer(container);
       verify(sched).markContainerForPreemption(appAttemptId, container);
-      verify(sched).markContainerForKillable(container);
+      verify(sched).killPreemptedContainer(container);
     } catch (InterruptedException e) {
       Assert.fail();
     } finally {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fa7a4352/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 3057615..028afb1 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
@@ -2352,7 +2352,7 @@ public class TestRMRestart extends ParameterizedSchedulerTestBase {
       FiCaSchedulerApp schedulerAppAttempt = cs.getSchedulerApplications()
           .get(app0.getApplicationId()).getCurrentAppAttempt();
       // kill app0-attempt
-      cs.markContainerForKillable(schedulerAppAttempt.getRMContainer(
+      cs.killPreemptedContainer(schedulerAppAttempt.getRMContainer(
           app0.getCurrentAppAttempt().getMasterContainer().getId()));
       am0.waitForState(RMAppAttemptState.FAILED);
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fa7a4352/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/applicationsmanager/TestAMRestart.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/TestAMRestart.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/applicationsmanager/TestAMRestart.java
index 16f3f60..5035afe 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/applicationsmanager/TestAMRestart.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/applicationsmanager/TestAMRestart.java
@@ -63,6 +63,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEv
 import org.apache.hadoop.yarn.server.utils.BuilderUtils;
 import org.apache.hadoop.yarn.util.ControlledClock;
 import org.apache.hadoop.yarn.util.Records;
+import org.apache.hadoop.yarn.util.SystemClock;
 import org.apache.hadoop.yarn.util.resource.Resources;
 import org.junit.Assert;
 import org.junit.Test;
@@ -565,7 +566,7 @@ public class TestAMRestart {
     ContainerId amContainer =
         ContainerId.newContainerId(am1.getApplicationAttemptId(), 1);
     // Preempt the first attempt;
-    scheduler.markContainerForKillable(scheduler.getRMContainer(amContainer));
+    scheduler.killPreemptedContainer(scheduler.getRMContainer(amContainer));
 
     am1.waitForState(RMAppAttemptState.FAILED);
     Assert.assertTrue(! attempt1.shouldCountTowardsMaxAttemptRetry());
@@ -581,7 +582,7 @@ public class TestAMRestart {
     // Preempt the second attempt.
     ContainerId amContainer2 =
         ContainerId.newContainerId(am2.getApplicationAttemptId(), 1);
-    scheduler.markContainerForKillable(scheduler.getRMContainer(amContainer2));
+    scheduler.killPreemptedContainer(scheduler.getRMContainer(amContainer2));
 
     am2.waitForState(RMAppAttemptState.FAILED);
     Assert.assertTrue(! attempt2.shouldCountTowardsMaxAttemptRetry());
@@ -676,7 +677,7 @@ public class TestAMRestart {
         ContainerId.newContainerId(am1.getApplicationAttemptId(), 1);
 
     // Forcibly preempt the am container;
-    scheduler.markContainerForKillable(scheduler.getRMContainer(amContainer));
+    scheduler.killPreemptedContainer(scheduler.getRMContainer(amContainer));
 
     am1.waitForState(RMAppAttemptState.FAILED);
     Assert.assertTrue(! attempt1.shouldCountTowardsMaxAttemptRetry());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fa7a4352/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/TestProportionalCapacityPreemptionPolicy.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/monitor/capacity/TestProportionalCapacityPreemptionPolicy.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/TestProportionalCapacityPreemptionPolicy.java
index e9129de..13f267d 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/TestProportionalCapacityPreemptionPolicy.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/TestProportionalCapacityPreemptionPolicy.java
@@ -23,7 +23,7 @@ import static org.apache.hadoop.yarn.server.resourcemanager.monitor.capacity.Pro
 import static org.apache.hadoop.yarn.server.resourcemanager.monitor.capacity.ProportionalCapacityPreemptionPolicy.OBSERVE_ONLY;
 import static org.apache.hadoop.yarn.server.resourcemanager.monitor.capacity.ProportionalCapacityPreemptionPolicy.TOTAL_PREEMPTION_PER_ROUND;
 import static org.apache.hadoop.yarn.server.resourcemanager.monitor.capacity.ProportionalCapacityPreemptionPolicy.WAIT_TIME_BEFORE_KILL;
-import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEventType.MARK_CONTAINER_FOR_KILLABLE;
+import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEventType.KILL_PREEMPTED_CONTAINER;
 import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEventType.MARK_CONTAINER_FOR_PREEMPTION;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNotNull;
@@ -75,7 +75,6 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.Capacity
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.LeafQueue;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.ParentQueue;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueueCapacities;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.preemption.PreemptionManager;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerApp;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.ContainerPreemptEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEvent;
@@ -168,7 +167,6 @@ public class TestProportionalCapacityPreemptionPolicy {
     when(mCS.getConfiguration()).thenReturn(schedConf);
     rmContext = mock(RMContext.class);
     when(mCS.getRMContext()).thenReturn(rmContext);
-    when(mCS.getPreemptionManager()).thenReturn(new PreemptionManager());
     when(rmContext.getNodeLabelManager()).thenReturn(lm);
     mDisp = mock(EventHandler.class);
     Dispatcher disp = mock(Dispatcher.class);
@@ -291,7 +289,7 @@ public class TestProportionalCapacityPreemptionPolicy {
     List<ContainerPreemptEvent> events = evtCaptor.getAllValues();
     for (ContainerPreemptEvent e : events.subList(20, 20)) {
       assertEquals(appC, e.getAppId());
-      assertEquals(MARK_CONTAINER_FOR_KILLABLE, e.getType());
+      assertEquals(KILL_PREEMPTED_CONTAINER, e.getType());
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fa7a4352/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/TestProportionalCapacityPreemptionPolicyForNodePartitions.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/monitor/capacity/TestProportionalCapacityPreemptionPolicyForNodePartitions.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/TestProportionalCapacityPreemptionPolicyForNodePartitions.java
index 21ea495..512f37c 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/TestProportionalCapacityPreemptionPolicyForNodePartitions.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/TestProportionalCapacityPreemptionPolicyForNodePartitions.java
@@ -67,7 +67,6 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.Capacity
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.LeafQueue;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.ParentQueue;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueueCapacities;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.preemption.PreemptionManager;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerApp;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.policy.OrderingPolicy;
@@ -124,7 +123,6 @@ public class TestProportionalCapacityPreemptionPolicyForNodePartitions {
     mClock = mock(Clock.class);
     cs = mock(CapacityScheduler.class);
     when(cs.getResourceCalculator()).thenReturn(rc);
-    when(cs.getPreemptionManager()).thenReturn(new PreemptionManager());
 
     nlm = mock(RMNodeLabelsManager.class);
     mDisp = mock(EventHandler.class);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fa7a4352/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 171196f..0b32676 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
@@ -52,7 +52,6 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttempt;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ActiveUsersManager;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceLimits;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.preemption.PreemptionManager;
 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.server.resourcemanager.security.RMContainerTokenSecretManager;
@@ -265,7 +264,6 @@ public class TestApplicationLimits {
         thenReturn(CapacityScheduler.nonPartitionedQueueComparator);
     when(csContext.getResourceCalculator()).thenReturn(resourceCalculator);
     when(csContext.getRMContext()).thenReturn(rmContext);
-    when(csContext.getPreemptionManager()).thenReturn(new PreemptionManager());
     
     // Say cluster has 100 nodes of 16G each
     Resource clusterResource = 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fa7a4352/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestApplicationPriority.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/TestApplicationPriority.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/TestApplicationPriority.java
index d8161f8..1569a12 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/TestApplicationPriority.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/TestApplicationPriority.java
@@ -205,7 +205,7 @@ public class TestApplicationPriority {
       if (++counter > 2) {
         break;
       }
-      cs.markContainerForKillable(schedulerAppAttempt.getRMContainer(c.getId()));
+      cs.killPreemptedContainer(schedulerAppAttempt.getRMContainer(c.getId()));
     }
 
     // check node report, 12 GB used and 4 GB available
@@ -512,7 +512,7 @@ public class TestApplicationPriority {
       if (++counter > 2) {
         break;
       }
-      cs.markContainerForKillable(schedulerAppAttemptApp1.getRMContainer(c.getId()));
+      cs.killPreemptedContainer(schedulerAppAttemptApp1.getRMContainer(c.getId()));
       iterator.remove();
     }
 
@@ -542,7 +542,7 @@ public class TestApplicationPriority {
       if (++counter > 1) {
         break;
       }
-      cs.markContainerForKillable(schedulerAppAttemptApp1.getRMContainer(c.getId()));
+      cs.killPreemptedContainer(schedulerAppAttemptApp1.getRMContainer(c.getId()));
       iterator.remove();
     }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fa7a4352/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacityScheduler.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/TestCapacityScheduler.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/TestCapacityScheduler.java
index 16ba607..b6c005b 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/TestCapacityScheduler.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/TestCapacityScheduler.java
@@ -1188,7 +1188,7 @@ public class TestCapacityScheduler {
 
     // kill the 3 containers
     for (Container c : allocatedContainers) {
-      cs.markContainerForKillable(schedulerAppAttempt.getRMContainer(c.getId()));
+      cs.killPreemptedContainer(schedulerAppAttempt.getRMContainer(c.getId()));
     }
 
     // check values
@@ -1197,7 +1197,7 @@ public class TestCapacityScheduler {
         Resource.newInstance(CONTAINER_MEMORY * 3, 3), false, 3);
 
     // kill app0-attempt0 AM container
-    cs.markContainerForKillable(schedulerAppAttempt.getRMContainer(app0
+    cs.killPreemptedContainer(schedulerAppAttempt.getRMContainer(app0
         .getCurrentAppAttempt().getMasterContainer().getId()));
 
     // wait for app0 failed
@@ -1220,7 +1220,7 @@ public class TestCapacityScheduler {
     allocatedContainers =
         am1.allocateAndWaitForContainers(3, CONTAINER_MEMORY, nm1);
     for (Container c : allocatedContainers) {
-      cs.markContainerForKillable(schedulerAppAttempt.getRMContainer(c.getId()));
+      cs.killPreemptedContainer(schedulerAppAttempt.getRMContainer(c.getId()));
     }
 
     // check values
@@ -1269,7 +1269,7 @@ public class TestCapacityScheduler {
     }
 
     // Call killContainer to preempt the container
-    cs.markContainerForKillable(rmContainer);
+    cs.killPreemptedContainer(rmContainer);
 
     Assert.assertEquals(3, requests.size());
     for (ResourceRequest request : requests) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fa7a4352/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacitySchedulerPreemption.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/TestCapacitySchedulerPreemption.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/TestCapacitySchedulerPreemption.java
deleted file mode 100644
index bea7797..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacitySchedulerPreemption.java
+++ /dev/null
@@ -1,677 +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.resourcemanager.scheduler.capacity;
-
-import com.google.common.collect.Sets;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.service.Service;
-import org.apache.hadoop.yarn.api.records.ContainerId;
-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.RMActiveServices;
-import org.apache.hadoop.yarn.server.resourcemanager.monitor.SchedulingEditPolicy;
-import org.apache.hadoop.yarn.server.resourcemanager.monitor.SchedulingMonitor;
-import org.apache.hadoop.yarn.server.resourcemanager.monitor.capacity.ProportionalCapacityPreemptionPolicy;
-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.rmnode.RMNode;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.preemption.PreemptionManager;
-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.util.Clock;
-import org.apache.hadoop.yarn.util.resource.Resources;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.Test;
-
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.HashSet;
-import java.util.Map;
-import java.util.Set;
-
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.when;
-
-public class TestCapacitySchedulerPreemption {
-  private static final Log LOG = LogFactory.getLog(
-      TestCapacitySchedulerPreemption.class);
-
-  private final int GB = 1024;
-
-  private Configuration conf;
-
-  RMNodeLabelsManager mgr;
-
-  Clock clock;
-
-  @Before
-  public void setUp() throws Exception {
-    conf = new YarnConfiguration();
-    conf.setClass(YarnConfiguration.RM_SCHEDULER, CapacityScheduler.class,
-        ResourceScheduler.class);
-    conf.setBoolean(YarnConfiguration.RM_SCHEDULER_ENABLE_MONITORS, true);
-    conf.setClass(YarnConfiguration.RM_SCHEDULER_MONITOR_POLICIES,
-        ProportionalCapacityPreemptionPolicy.class, SchedulingEditPolicy.class);
-    conf = TestUtils.getConfigurationWithMultipleQueues(this.conf);
-
-    // Set preemption related configurations
-    conf.setInt(ProportionalCapacityPreemptionPolicy.WAIT_TIME_BEFORE_KILL,
-        0);
-    conf.setBoolean(CapacitySchedulerConfiguration.LAZY_PREEMPTION_ENALBED,
-        true);
-    conf.setFloat(
-        ProportionalCapacityPreemptionPolicy.TOTAL_PREEMPTION_PER_ROUND, 1.0f);
-    conf.setFloat(
-        ProportionalCapacityPreemptionPolicy.NATURAL_TERMINATION_FACTOR, 1.0f);
-    mgr = new NullRMNodeLabelsManager();
-    mgr.init(this.conf);
-    clock = mock(Clock.class);
-    when(clock.getTime()).thenReturn(0L);
-  }
-
-  private SchedulingEditPolicy getSchedulingEditPolicy(MockRM rm) {
-    RMActiveServices activeServices = rm.getRMActiveService();
-    SchedulingMonitor mon = null;
-    for (Service service : activeServices.getServices()) {
-      if (service instanceof SchedulingMonitor) {
-        mon = (SchedulingMonitor) service;
-        break;
-      }
-    }
-
-    if (mon != null) {
-      return mon.getSchedulingEditPolicy();
-    }
-    return null;
-  }
-
-  @Test (timeout = 60000)
-  public void testSimplePreemption() throws Exception {
-    /**
-     * Test case: Submit two application (app1/app2) to different queues, queue
-     * structure:
-     *
-     * <pre>
-     *             Root
-     *            /  |  \
-     *           a   b   c
-     *          10   20  70
-     * </pre>
-     *
-     * 1) Two nodes in the cluster, each of them has 4G.
-     *
-     * 2) app1 submit to queue-a first, it asked 7 * 1G containers, so there's no
-     * more resource available.
-     *
-     * 3) app2 submit to queue-c, ask for one 1G container (for AM)
-     *
-     * Now the cluster is fulfilled.
-     *
-     * 4) app2 asks for another 1G container, system will preempt one container
-     * from app1, and app2 will receive the preempted container
-     */
-    MockRM rm1 = new MockRM(conf);
-    rm1.getRMContext().setNodeLabelManager(mgr);
-    rm1.start();
-
-    MockNM nm1 = rm1.registerNode("h1:1234", 4 * GB);
-    MockNM nm2 = rm1.registerNode("h2:1234", 4 * GB);
-    CapacityScheduler cs = (CapacityScheduler) rm1.getResourceScheduler();
-    RMNode rmNode1 = rm1.getRMContext().getRMNodes().get(nm1.getNodeId());
-    RMNode rmNode2 = rm1.getRMContext().getRMNodes().get(nm2.getNodeId());
-
-    // launch an app to queue, AM container should be launched in nm1
-    RMApp app1 = rm1.submitApp(1 * GB, "app", "user", null, "a");
-    MockAM am1 = MockRM.launchAndRegisterAM(app1, rm1, nm1);
-
-    am1.allocate("*", 1 * GB, 7, new ArrayList<ContainerId>());
-
-    // Do allocation 3 times for node1/node2
-    for (int i = 0; i < 3; i++) {
-      cs.handle(new NodeUpdateSchedulerEvent(rmNode1));
-      cs.handle(new NodeUpdateSchedulerEvent(rmNode2));
-    }
-
-    // App1 should have 7 containers now, and no available resource for cluster
-    FiCaSchedulerApp schedulerApp1 = cs.getApplicationAttempt(
-        am1.getApplicationAttemptId());
-    Assert.assertEquals(7, schedulerApp1.getLiveContainers().size());
-
-    // Submit app2 to queue-c and asks for a 1G container for AM
-    RMApp app2 = rm1.submitApp(1 * GB, "app", "user", null, "c");
-    MockAM am2 = MockRM.launchAndRegisterAM(app2, rm1, nm2);
-
-    // NM1/NM2 has available resource = 0G
-    Assert.assertEquals(0 * GB, cs.getNode(nm1.getNodeId())
-        .getUnallocatedResource().getMemory());
-    Assert.assertEquals(0 * GB, cs.getNode(nm2.getNodeId())
-        .getUnallocatedResource().getMemory());
-
-    // AM asks for a 1 * GB container
-    am2.allocate(Arrays.asList(ResourceRequest
-        .newInstance(Priority.newInstance(1), ResourceRequest.ANY,
-            Resources.createResource(1 * GB), 1)), null);
-
-    // Get edit policy and do one update
-    SchedulingEditPolicy editPolicy = getSchedulingEditPolicy(rm1);
-
-    // Call edit schedule twice, and check if one container from app1 marked
-    // to be "killable"
-    editPolicy.editSchedule();
-    editPolicy.editSchedule();
-
-    PreemptionManager pm = cs.getPreemptionManager();
-    Map<ContainerId, RMContainer> killableContainers =
-        waitKillableContainersSize(pm, "a", RMNodeLabelsManager.NO_LABEL, 1);
-    Assert.assertEquals(1, killableContainers.size());
-    Assert.assertEquals(killableContainers.entrySet().iterator().next().getKey()
-        .getApplicationAttemptId(), am1.getApplicationAttemptId());
-
-    // Call CS.handle once to see if container preempted
-    cs.handle(new NodeUpdateSchedulerEvent(rmNode2));
-
-    FiCaSchedulerApp schedulerApp2 = cs.getApplicationAttempt(
-        am2.getApplicationAttemptId());
-
-    // App1 has 6 containers, and app2 has 2 containers
-    Assert.assertEquals(6, schedulerApp1.getLiveContainers().size());
-    Assert.assertEquals(2, schedulerApp2.getLiveContainers().size());
-
-    rm1.close();
-  }
-
-  @Test (timeout = 60000)
-  public void testPreemptionConsidersNodeLocalityDelay()
-      throws Exception {
-    /**
-     * Test case: same as testSimplePreemption steps 1-3.
-     *
-     * Step 4: app2 asks for 1G container with locality specified, so it needs
-     * to wait for missed-opportunity before get scheduled.
-     * Check if system waits missed-opportunity before finish killable container
-     */
-    MockRM rm1 = new MockRM(conf);
-    rm1.getRMContext().setNodeLabelManager(mgr);
-    rm1.start();
-    MockNM nm1 = rm1.registerNode("h1:1234", 4 * GB);
-    MockNM nm2 = rm1.registerNode("h2:1234", 4 * GB);
-    CapacityScheduler cs = (CapacityScheduler) rm1.getResourceScheduler();
-    RMNode rmNode1 = rm1.getRMContext().getRMNodes().get(nm1.getNodeId());
-    RMNode rmNode2 = rm1.getRMContext().getRMNodes().get(nm2.getNodeId());
-
-    // launch an app to queue, AM container should be launched in nm1
-    RMApp app1 = rm1.submitApp(1 * GB, "app", "user", null, "a");
-    MockAM am1 = MockRM.launchAndRegisterAM(app1, rm1, nm1);
-
-    am1.allocate("*", 1 * GB, 6, new ArrayList<ContainerId>());
-
-    // Do allocation 3 times for node1/node2
-    for (int i = 0; i < 3; i++) {
-      cs.handle(new NodeUpdateSchedulerEvent(rmNode1));
-      cs.handle(new NodeUpdateSchedulerEvent(rmNode2));
-    }
-
-    // App1 should have 7 containers now, and no available resource for cluster
-    FiCaSchedulerApp schedulerApp1 = cs.getApplicationAttempt(
-        am1.getApplicationAttemptId());
-    Assert.assertEquals(7, schedulerApp1.getLiveContainers().size());
-
-    // Submit app2 to queue-c and asks for a 1G container for AM
-    RMApp app2 = rm1.submitApp(1 * GB, "app", "user", null, "c");
-    MockAM am2 = MockRM.launchAndRegisterAM(app2, rm1, nm2);
-
-    // NM1/NM2 has available resource = 0G
-    Assert.assertEquals(0 * GB, cs.getNode(nm1.getNodeId())
-        .getUnallocatedResource().getMemory());
-    Assert.assertEquals(0 * GB, cs.getNode(nm2.getNodeId())
-        .getUnallocatedResource().getMemory());
-
-    // AM asks for a 1 * GB container with unknown host and unknown rack
-    am2.allocate(Arrays.asList(ResourceRequest
-        .newInstance(Priority.newInstance(1), ResourceRequest.ANY,
-            Resources.createResource(1 * GB), 1), ResourceRequest
-        .newInstance(Priority.newInstance(1), "unknownhost",
-            Resources.createResource(1 * GB), 1), ResourceRequest
-        .newInstance(Priority.newInstance(1), "/default-rack",
-            Resources.createResource(1 * GB), 1)), null);
-
-    // Get edit policy and do one update
-    SchedulingEditPolicy editPolicy = getSchedulingEditPolicy(rm1);
-
-    // Call edit schedule twice, and check if one container from app1 marked
-    // to be "killable"
-    editPolicy.editSchedule();
-    editPolicy.editSchedule();
-
-    PreemptionManager pm = cs.getPreemptionManager();
-    Map<ContainerId, RMContainer> killableContainers =
-        waitKillableContainersSize(pm, "a", RMNodeLabelsManager.NO_LABEL, 1);
-    Assert.assertEquals(killableContainers.entrySet().iterator().next().getKey()
-        .getApplicationAttemptId(), am1.getApplicationAttemptId());
-
-    // Call CS.handle once to see if container preempted
-    cs.handle(new NodeUpdateSchedulerEvent(rmNode2));
-
-    FiCaSchedulerApp schedulerApp2 = cs.getApplicationAttempt(
-        am2.getApplicationAttemptId());
-
-    // App1 has 7 containers, and app2 has 1 containers (no container preempted)
-    Assert.assertEquals(7, schedulerApp1.getLiveContainers().size());
-    Assert.assertEquals(1, schedulerApp2.getLiveContainers().size());
-
-    // Do allocation again, one container will be preempted
-    cs.handle(new NodeUpdateSchedulerEvent(rmNode2));
-
-    // App1 has 6 containers, and app2 has 2 containers (new container allocated)
-    Assert.assertEquals(6, schedulerApp1.getLiveContainers().size());
-    Assert.assertEquals(2, schedulerApp2.getLiveContainers().size());
-
-    rm1.close();
-  }
-
-  @Test (timeout = 60000)
-  public void testPreemptionConsidersHardNodeLocality()
-      throws Exception {
-    /**
-     * Test case: same as testSimplePreemption steps 1-3.
-     *
-     * Step 4: app2 asks for 1G container with hard locality specified, and
-     *         asked host is not existed
-     * Confirm system doesn't preempt any container.
-     */
-    MockRM rm1 = new MockRM(conf);
-    rm1.getRMContext().setNodeLabelManager(mgr);
-    rm1.start();
-    MockNM nm1 = rm1.registerNode("h1:1234", 4 * GB);
-    MockNM nm2 = rm1.registerNode("h2:1234", 4 * GB);
-    CapacityScheduler cs = (CapacityScheduler) rm1.getResourceScheduler();
-    RMNode rmNode1 = rm1.getRMContext().getRMNodes().get(nm1.getNodeId());
-    RMNode rmNode2 = rm1.getRMContext().getRMNodes().get(nm2.getNodeId());
-
-    // launch an app to queue, AM container should be launched in nm1
-    RMApp app1 = rm1.submitApp(1 * GB, "app", "user", null, "a");
-    MockAM am1 = MockRM.launchAndRegisterAM(app1, rm1, nm1);
-
-    am1.allocate("*", 1 * GB, 6, new ArrayList<ContainerId>());
-
-    // Do allocation 3 times for node1/node2
-    for (int i = 0; i < 3; i++) {
-      cs.handle(new NodeUpdateSchedulerEvent(rmNode1));
-    }
-    for (int i = 0; i < 3; i++) {
-      cs.handle(new NodeUpdateSchedulerEvent(rmNode2));
-    }
-
-    // App1 should have 7 containers now, and no available resource for cluster
-    FiCaSchedulerApp schedulerApp1 = cs.getApplicationAttempt(
-        am1.getApplicationAttemptId());
-    Assert.assertEquals(7, schedulerApp1.getLiveContainers().size());
-
-    // Submit app2 to queue-c and asks for a 1G container for AM
-    RMApp app2 = rm1.submitApp(1 * GB, "app", "user", null, "c");
-    MockAM am2 = MockRM.launchAndRegisterAM(app2, rm1, nm2);
-
-    // NM1/NM2 has available resource = 0G
-    Assert.assertEquals(0 * GB, cs.getNode(nm1.getNodeId())
-        .getUnallocatedResource().getMemory());
-    Assert.assertEquals(0 * GB, cs.getNode(nm2.getNodeId())
-        .getUnallocatedResource().getMemory());
-
-    // AM asks for a 1 * GB container for h3 with hard locality,
-    // h3 doesn't exist in the cluster
-    am2.allocate(Arrays.asList(ResourceRequest
-        .newInstance(Priority.newInstance(1), ResourceRequest.ANY,
-            Resources.createResource(1 * GB), 1, true), ResourceRequest
-        .newInstance(Priority.newInstance(1), "h3",
-            Resources.createResource(1 * GB), 1, false), ResourceRequest
-        .newInstance(Priority.newInstance(1), "/default-rack",
-            Resources.createResource(1 * GB), 1, false)), null);
-
-    // Get edit policy and do one update
-    SchedulingEditPolicy editPolicy = getSchedulingEditPolicy(rm1);
-
-    // Call edit schedule twice, and check if one container from app1 marked
-    // to be "killable"
-    editPolicy.editSchedule();
-    editPolicy.editSchedule();
-
-    PreemptionManager pm = cs.getPreemptionManager();
-    Map<ContainerId, RMContainer> killableContainers =
-        waitKillableContainersSize(pm, "a", RMNodeLabelsManager.NO_LABEL, 1);
-    Assert.assertEquals(killableContainers.entrySet().iterator().next().getKey()
-        .getApplicationAttemptId(), am1.getApplicationAttemptId());
-
-    // Call CS.handle once to see if container preempted
-    cs.handle(new NodeUpdateSchedulerEvent(rmNode2));
-
-    FiCaSchedulerApp schedulerApp2 = cs.getApplicationAttempt(
-        am2.getApplicationAttemptId());
-
-    // App1 has 7 containers, and app2 has 1 containers (no container preempted)
-    Assert.assertEquals(7, schedulerApp1.getLiveContainers().size());
-    Assert.assertEquals(1, schedulerApp2.getLiveContainers().size());
-
-    // Do allocation again, nothing will be preempted
-    cs.handle(new NodeUpdateSchedulerEvent(rmNode2));
-
-    // App1 has 7 containers, and app2 has 1 containers (no container allocated)
-    Assert.assertEquals(7, schedulerApp1.getLiveContainers().size());
-    Assert.assertEquals(1, schedulerApp2.getLiveContainers().size());
-
-    rm1.close();
-  }
-
-  @Test (timeout = 60000)
-  public void testPreemptionPolicyShouldRespectAlreadyMarkedKillableContainers()
-      throws Exception {
-    /**
-     * Test case:
-     * <pre>
-     *             Root
-     *            /  |  \
-     *           a   b   c
-     *          10   20  70
-     * </pre>
-     * Submit applications to two queues, one uses more than the other, so
-     * preemption will happen.
-     *
-     * Check:
-     * 1) Killable containers resources will be excluded from PCPP (no duplicated
-     *    container added to killable list)
-     * 2) When more resources need to be preempted, new containers will be selected
-     *    and killable containers will be considered
-     */
-    MockRM rm1 = new MockRM(conf);
-    rm1.getRMContext().setNodeLabelManager(mgr);
-    rm1.start();
-    MockNM nm1 = rm1.registerNode("h1:1234", 8 * GB);
-    CapacityScheduler cs = (CapacityScheduler) rm1.getResourceScheduler();
-    RMNode rmNode1 = rm1.getRMContext().getRMNodes().get(nm1.getNodeId());
-
-    // launch an app to queue, AM container should be launched in nm1
-    RMApp app1 = rm1.submitApp(1 * GB, "app", "user", null, "a");
-    MockAM am1 = MockRM.launchAndRegisterAM(app1, rm1, nm1);
-
-    am1.allocate("*", 1 * GB, 6, new ArrayList<ContainerId>());
-
-    // Do allocation 6 times for node1
-    for (int i = 0; i < 6; i++) {
-      cs.handle(new NodeUpdateSchedulerEvent(rmNode1));
-    }
-
-    // App1 should have 7 containers now, and no available resource for cluster
-    FiCaSchedulerApp schedulerApp1 = cs.getApplicationAttempt(
-        am1.getApplicationAttemptId());
-    Assert.assertEquals(7, schedulerApp1.getLiveContainers().size());
-
-    // Submit app2 to queue-c and asks for a 1G container for AM
-    RMApp app2 = rm1.submitApp(1 * GB, "app", "user", null, "c");
-    MockAM am2 = MockRM.launchAndRegisterAM(app2, rm1, nm1);
-
-    // NM1 has available resource = 0G
-    Assert.assertEquals(0 * GB, cs.getNode(nm1.getNodeId())
-        .getUnallocatedResource().getMemory());
-    am2.allocate("*", 1 * GB, 1, new ArrayList<ContainerId>());
-
-    // Get edit policy and do one update
-    ProportionalCapacityPreemptionPolicy editPolicy =
-        (ProportionalCapacityPreemptionPolicy) getSchedulingEditPolicy(rm1);
-
-    // Call edit schedule twice, and check if one container from app1 marked
-    // to be "killable"
-    editPolicy.editSchedule();
-    editPolicy.editSchedule();
-
-    PreemptionManager pm = cs.getPreemptionManager();
-    waitKillableContainersSize(pm, "a", RMNodeLabelsManager.NO_LABEL, 1);
-
-    // Check killable containers and to-be-preempted containers in edit policy
-    Assert.assertEquals(0, editPolicy.getToPreemptContainers().size());
-
-    // Run edit schedule again, confirm status doesn't changed
-    editPolicy.editSchedule();
-    Assert.assertEquals(0, editPolicy.getToPreemptContainers().size());
-
-    // Save current to kill containers
-    Set<ContainerId> previousKillableContainers = new HashSet<>(
-        pm.getKillableContainersMap("a", RMNodeLabelsManager.NO_LABEL)
-            .keySet());
-
-    // Update request resource of c from 1 to 2, so we need to preempt
-    // one more container
-    am2.allocate("*", 1 * GB, 2, new ArrayList<ContainerId>());
-
-    // Call editPolicy.editSchedule() once, we should have 1 container in to-preempt map
-    // and 1 container in killable map
-    editPolicy.editSchedule();
-    Assert.assertEquals(1, editPolicy.getToPreemptContainers().size());
-
-    // Call editPolicy.editSchedule() once more, we should have 2 containers killable map
-    editPolicy.editSchedule();
-    Assert.assertEquals(0, editPolicy.getToPreemptContainers().size());
-
-    // Check if previous killable containers included by new killable containers
-    Map<ContainerId, RMContainer> killableContainers =
-        waitKillableContainersSize(pm, "a", RMNodeLabelsManager.NO_LABEL, 2);
-    Assert.assertTrue(
-        Sets.difference(previousKillableContainers, killableContainers.keySet())
-            .isEmpty());
-  }
-
-  @Test (timeout = 60000)
-  public void testPreemptionPolicyCleanupKillableContainersWhenNoPreemptionNeeded()
-      throws Exception {
-    /**
-     * Test case:
-     * <pre>
-     *             Root
-     *            /  |  \
-     *           a   b   c
-     *          10   20  70
-     * </pre>
-     * Submit applications to two queues, one uses more than the other, so
-     * preemption will happen.
-     *
-     * Check:
-     * 1) Containers will be marked to killable
-     * 2) Cancel resource request
-     * 3) Killable containers will be cancelled from policy and scheduler
-     */
-    MockRM rm1 = new MockRM(conf);
-    rm1.getRMContext().setNodeLabelManager(mgr);
-    rm1.start();
-    MockNM nm1 = rm1.registerNode("h1:1234", 8 * GB);
-    CapacityScheduler cs = (CapacityScheduler) rm1.getResourceScheduler();
-    RMNode rmNode1 = rm1.getRMContext().getRMNodes().get(nm1.getNodeId());
-
-    // launch an app to queue, AM container should be launched in nm1
-    RMApp app1 = rm1.submitApp(1 * GB, "app", "user", null, "a");
-    MockAM am1 = MockRM.launchAndRegisterAM(app1, rm1, nm1);
-
-    am1.allocate("*", 1 * GB, 6, new ArrayList<ContainerId>());
-
-    // Do allocation 6 times for node1
-    for (int i = 0; i < 6; i++) {
-      cs.handle(new NodeUpdateSchedulerEvent(rmNode1));
-    }
-
-    // App1 should have 7 containers now, and no available resource for cluster
-    FiCaSchedulerApp schedulerApp1 = cs.getApplicationAttempt(
-        am1.getApplicationAttemptId());
-    Assert.assertEquals(7, schedulerApp1.getLiveContainers().size());
-
-    // Submit app2 to queue-c and asks for a 1G container for AM
-    RMApp app2 = rm1.submitApp(1 * GB, "app", "user", null, "c");
-    MockAM am2 = MockRM.launchAndRegisterAM(app2, rm1, nm1);
-
-    // NM1 has available resource = 0G
-    Assert.assertEquals(0 * GB, cs.getNode(nm1.getNodeId())
-        .getUnallocatedResource().getMemory());
-    am2.allocate("*", 3 * GB, 1, new ArrayList<ContainerId>());
-
-    // Get edit policy and do one update
-    ProportionalCapacityPreemptionPolicy editPolicy =
-        (ProportionalCapacityPreemptionPolicy) getSchedulingEditPolicy(rm1);
-
-    // Call edit schedule twice, and check if 3 container from app1 marked
-    // to be "killable"
-    editPolicy.editSchedule();
-    editPolicy.editSchedule();
-
-    PreemptionManager pm = cs.getPreemptionManager();
-    waitKillableContainersSize(pm, "a", RMNodeLabelsManager.NO_LABEL, 3);
-
-    // Change reqeust from 3G to 2G, now we can preempt one less container. (3->2)
-    am2.allocate("*", 2 * GB, 1, new ArrayList<ContainerId>());
-    editPolicy.editSchedule();
-    Assert.assertEquals(0, editPolicy.getToPreemptContainers().size());
-    waitKillableContainersSize(pm, "a", RMNodeLabelsManager.NO_LABEL, 2);
-
-    // Call editSchedule once more to make sure still nothing happens
-    editPolicy.editSchedule();
-    Assert.assertEquals(0, editPolicy.getToPreemptContainers().size());
-    waitKillableContainersSize(pm, "a", RMNodeLabelsManager.NO_LABEL, 2);
-  }
-
-  @Test (timeout = 60000)
-  public void testPreemptionConsidersUserLimit()
-      throws Exception {
-    /**
-     * Test case: Submit two application (app1/app2) to different queues, queue
-     * structure:
-     *
-     * <pre>
-     *             Root
-     *            /  |  \
-     *           a   b   c
-     *          10   20  70
-     * </pre>
-     *
-     * Queue-c's user-limit-factor = 0.1, so single user cannot allocate >1 containers in queue-c
-     *
-     * 1) Two nodes in the cluster, each of them has 4G.
-     *
-     * 2) app1 submit to queue-a first, it asked 7 * 1G containers, so there's no
-     * more resource available.
-     *
-     * 3) app2 submit to queue-c, ask for one 1G container (for AM)
-     *
-     * Now the cluster is fulfilled.
-     *
-     * 4) app2 asks for another 1G container, system will preempt one container
-     * from app1, and app2 will receive the preempted container
-     */
-    CapacitySchedulerConfiguration csConf = new CapacitySchedulerConfiguration(conf);
-    csConf.setUserLimitFactor(CapacitySchedulerConfiguration.ROOT + ".c", 0.1f);
-    MockRM rm1 = new MockRM(csConf);
-    rm1.getRMContext().setNodeLabelManager(mgr);
-    rm1.start();
-    MockNM nm1 = rm1.registerNode("h1:1234", 4 * GB);
-    MockNM nm2 = rm1.registerNode("h2:1234", 4 * GB);
-    CapacityScheduler cs = (CapacityScheduler) rm1.getResourceScheduler();
-    RMNode rmNode1 = rm1.getRMContext().getRMNodes().get(nm1.getNodeId());
-    RMNode rmNode2 = rm1.getRMContext().getRMNodes().get(nm2.getNodeId());
-
-    // launch an app to queue, AM container should be launched in nm1
-    RMApp app1 = rm1.submitApp(1 * GB, "app", "user", null, "a");
-    MockAM am1 = MockRM.launchAndRegisterAM(app1, rm1, nm1);
-
-    am1.allocate("*", 1 * GB, 6, new ArrayList<ContainerId>());
-
-    // Do allocation 3 times for node1/node2
-    for (int i = 0; i < 3; i++) {
-      cs.handle(new NodeUpdateSchedulerEvent(rmNode1));
-      cs.handle(new NodeUpdateSchedulerEvent(rmNode2));
-    }
-
-    // App1 should have 7 containers now, and no available resource for cluster
-    FiCaSchedulerApp schedulerApp1 = cs.getApplicationAttempt(
-        am1.getApplicationAttemptId());
-    Assert.assertEquals(7, schedulerApp1.getLiveContainers().size());
-
-    // Submit app2 to queue-c and asks for a 1G container for AM
-    RMApp app2 = rm1.submitApp(1 * GB, "app", "user", null, "c");
-    MockAM am2 = MockRM.launchAndRegisterAM(app2, rm1, nm2);
-
-    // NM1/NM2 has available resource = 0G
-    Assert.assertEquals(0 * GB, cs.getNode(nm1.getNodeId())
-        .getUnallocatedResource().getMemory());
-    Assert.assertEquals(0 * GB, cs.getNode(nm2.getNodeId())
-        .getUnallocatedResource().getMemory());
-
-    // AM asks for a 1 * GB container
-    am2.allocate(Arrays.asList(ResourceRequest
-        .newInstance(Priority.newInstance(1), ResourceRequest.ANY,
-            Resources.createResource(1 * GB), 1)), null);
-
-    // Get edit policy and do one update
-    SchedulingEditPolicy editPolicy = getSchedulingEditPolicy(rm1);
-
-    // Call edit schedule twice, and check if no container from app1 marked
-    // to be "killable"
-    editPolicy.editSchedule();
-    editPolicy.editSchedule();
-
-    // No preemption happens
-    PreemptionManager pm = cs.getPreemptionManager();
-    Map<ContainerId, RMContainer> killableContainers =
-        waitKillableContainersSize(pm, "a", RMNodeLabelsManager.NO_LABEL, 0);
-    Assert.assertEquals(0, killableContainers.size());
-
-    // Call CS.handle once to see if container preempted
-    cs.handle(new NodeUpdateSchedulerEvent(rmNode2));
-
-    FiCaSchedulerApp schedulerApp2 = cs.getApplicationAttempt(
-        am2.getApplicationAttemptId());
-
-    // App1 has 7 containers, and app2 has 1 containers (nothing preempted)
-    Assert.assertEquals(7, schedulerApp1.getLiveContainers().size());
-    Assert.assertEquals(1, schedulerApp2.getLiveContainers().size());
-
-    rm1.close();
-  }
-
-  private Map<ContainerId, RMContainer> waitKillableContainersSize(
-      PreemptionManager pm, String queueName, String partition,
-      int expectedSize) throws InterruptedException {
-    Map<ContainerId, RMContainer> killableContainers =
-        pm.getKillableContainersMap(queueName, partition);
-
-    int wait = 0;
-    // Wait for at most 5 sec (it should be super fast actually)
-    while (expectedSize != killableContainers.size() && wait < 500) {
-      killableContainers = pm.getKillableContainersMap(queueName, partition);
-      Thread.sleep(10);
-      wait++;
-    }
-
-    Assert.assertEquals(expectedSize, killableContainers.size());
-    return killableContainers;
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fa7a4352/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 1612201..5169337 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
@@ -51,7 +51,6 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerEven
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerImpl;
 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.capacity.preemption.PreemptionManager;
 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.server.utils.BuilderUtils;
@@ -100,7 +99,6 @@ public class TestChildQueueOrder {
     when(csContext.getResourceCalculator()).
     thenReturn(resourceComparator);
     when(csContext.getRMContext()).thenReturn(rmContext);
-    when(csContext.getPreemptionManager()).thenReturn(new PreemptionManager());
   }
 
   private FiCaSchedulerApp getMockApplication(int appId, String user) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fa7a4352/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 87a3d51..69b0813 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
@@ -71,7 +71,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.QueueMetrics;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceLimits;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.preemption.PreemptionManager;
 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.server.resourcemanager.scheduler.event.AppAddedSchedulerEvent;
@@ -151,7 +150,6 @@ public class TestLeafQueue {
         thenReturn(CapacityScheduler.nonPartitionedQueueComparator);
     when(csContext.getResourceCalculator()).
         thenReturn(resourceCalculator);
-    when(csContext.getPreemptionManager()).thenReturn(new PreemptionManager());
     when(csContext.getRMContext()).thenReturn(rmContext);
     RMContainerTokenSecretManager containerTokenSecretManager =
         new RMContainerTokenSecretManager(conf);
@@ -3094,7 +3092,6 @@ public class TestLeafQueue {
         Resources.createResource(GB, 1));
     when(csContext.getMaximumResourceCapability()).thenReturn(
         Resources.createResource(2 * GB, 2));
-    when(csContext.getPreemptionManager()).thenReturn(new PreemptionManager());
     return csContext;
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fa7a4352/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 1ee201d..bbf6e43 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
@@ -39,7 +39,6 @@ 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.recovery.MemoryRMStateStore;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptState;
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
@@ -1677,100 +1676,4 @@ public class TestNodeLabelContainerAllocation {
     checkNumOfContainersInAnAppOnGivenNode(2, nm1.getNodeId(),
         cs.getApplicationAttempt(am1.getApplicationAttemptId()));
   }
-
-  @Test
-  public void testParentQueueMaxCapsAreRespected() throws Exception {
-    /*
-     * Queue tree:
-     *          Root
-     *        /     \
-     *       A       B
-     *      / \
-     *     A1 A2
-     *
-     * A has 50% capacity and 50% max capacity (of label=x)
-     * A1/A2 has 50% capacity and 100% max capacity (of label=x)
-     * Cluster has one node (label=x) with resource = 24G.
-     * So we can at most use 12G resources under queueA.
-     */
-    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, 10);
-    csConf.setAccessibleNodeLabels(A, toSet("x"));
-    csConf.setCapacityByLabel(A, "x", 50);
-    csConf.setMaximumCapacityByLabel(A, "x", 50);
-
-    final String B = CapacitySchedulerConfiguration.ROOT + ".b";
-    csConf.setCapacity(B, 90);
-    csConf.setAccessibleNodeLabels(B, toSet("x"));
-    csConf.setCapacityByLabel(B, "x", 50);
-    csConf.setMaximumCapacityByLabel(B, "x", 50);
-
-    // Define 2nd-level queues
-    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);
-    csConf.setMaximumCapacityByLabel(A1, "x", 100);
-    csConf.setUserLimitFactor(A1, 100.0f);
-
-    final String A2 = A + ".a2";
-    csConf.setCapacity(A2, 50);
-    csConf.setAccessibleNodeLabels(A2, toSet("x"));
-    csConf.setCapacityByLabel(A2, "x", 50);
-    csConf.setMaximumCapacityByLabel(A2, "x", 100);
-    csConf.setUserLimitFactor(A2, 100.0f);
-
-    // set node -> label
-    mgr.addToCluserNodeLabels(ImmutableSet.of(
-        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 =
-        new MockNM("h1:1234", 24 * GB, rm.getResourceTrackerService());
-    nm1.registerNode();
-
-    // Launch app1 in a1, resource usage is 1GB (am) + 4GB * 2 = 9GB
-    RMApp app1 = rm.submitApp(1 * GB, "app", "user", null, "a1", "x");
-    MockAM am1 = MockRM.launchAndRegisterAM(app1, rm, nm1);
-    am1.allocate("*", 4 * GB, 2, new ArrayList<ContainerId>(), "x");
-    doNMHeartbeat(rm, nm1.getNodeId(), 10);
-    checkNumOfContainersInAnAppOnGivenNode(3, nm1.getNodeId(),
-        cs.getApplicationAttempt(am1.getApplicationAttemptId()));
-
-    // Try to launch app2 in a2, asked 2GB, should success
-    RMApp app2 = rm.submitApp(2 * GB, "app", "user", null, "a2", "x");
-    MockAM am2 = MockRM.launchAndRegisterAM(app2, rm, nm1);
-
-    // am2 asks more resources, cannot success because current used = 9G (app1)
-    // + 2G (app2) = 11G, and queue's max capacity = 12G
-    am2.allocate("*", 2 * GB, 2, new ArrayList<ContainerId>(), "x");
-
-    doNMHeartbeat(rm, nm1.getNodeId(), 10);
-    checkNumOfContainersInAnAppOnGivenNode(1, nm1.getNodeId(),
-        cs.getApplicationAttempt(am2.getApplicationAttemptId()));
-  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fa7a4352/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 23dc860..f73baa4 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
@@ -47,7 +47,6 @@ 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.capacity.preemption.PreemptionManager;
 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.DefaultResourceCalculator;
@@ -93,7 +92,6 @@ public class TestParentQueue {
         thenReturn(Resources.createResource(100 * 16 * GB, 100 * 32));
     when(csContext.getNonPartitionedQueueComparator()).
     thenReturn(CapacityScheduler.nonPartitionedQueueComparator);
-    when(csContext.getPreemptionManager()).thenReturn(new PreemptionManager());
     when(csContext.getResourceCalculator()).
     thenReturn(resourceComparator);
     when(csContext.getRMContext()).thenReturn(rmContext);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fa7a4352/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 56facee..2ef5e39 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
@@ -55,7 +55,6 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerImpl
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ActiveUsersManager;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceLimits;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplicationAttempt.AMState;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.preemption.PreemptionManager;
 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.server.resourcemanager.security.RMContainerTokenSecretManager;
@@ -127,7 +126,6 @@ public class TestReservations {
     when(csContext.getNonPartitionedQueueComparator()).thenReturn(
         CapacityScheduler.nonPartitionedQueueComparator);
     when(csContext.getResourceCalculator()).thenReturn(resourceCalculator);
-    when(csContext.getPreemptionManager()).thenReturn(new PreemptionManager());
     when(csContext.getRMContext()).thenReturn(rmContext);
     RMContainerTokenSecretManager containerTokenSecretManager = new RMContainerTokenSecretManager(
         conf);


[02/46] hadoop git commit: MAPREDUCE-6520. Migrate MR Client test cases part 1.

Posted by ae...@apache.org.
MAPREDUCE-6520. Migrate MR Client test cases part 1.


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

Branch: refs/heads/HDFS-7240
Commit: ef68b441957c6deb65adae2272d60ac929d00733
Parents: 682adc6
Author: Akira Ajisaka <aa...@apache.org>
Authored: Fri Mar 11 22:20:55 2016 +0900
Committer: Akira Ajisaka <aa...@apache.org>
Committed: Fri Mar 11 22:20:55 2016 +0900

----------------------------------------------------------------------
 .../hadoop/conf/TestNoDefaultsJobConf.java      | 30 +++++++++++----
 .../apache/hadoop/mapred/HadoopTestCase.java    | 17 ++++-----
 .../hadoop/mapred/NotificationTestCase.java     | 15 ++++++--
 .../hadoop/mapred/TestFileOutputFormat.java     |  4 ++
 .../apache/hadoop/mapred/TestTaskCommit.java    | 31 ++++++++++-----
 .../mapred/jobcontrol/TestLocalJobControl.java  |  3 ++
 .../hadoop/mapred/lib/TestChainMapReduce.java   | 20 +++++++++-
 .../mapred/lib/TestKeyFieldBasedComparator.java |  2 +
 .../hadoop/mapred/lib/TestMultipleInputs.java   | 17 +++++----
 .../hadoop/mapred/lib/TestMultipleOutputs.java  | 26 ++++++++++++-
 .../mapred/lib/TestMultithreadedMapRunner.java  | 21 +++++++++-
 .../org/apache/hadoop/mapreduce/TestChild.java  | 10 ++++-
 .../hadoop/mapreduce/TestNoJobSetupCleanup.java |  6 ++-
 .../hadoop/mapreduce/TestTaskContext.java       |  2 +
 .../mapreduce/lib/chain/TestChainErrors.java    |  9 +++++
 .../mapreduce/lib/chain/TestMapReduceChain.java |  6 +++
 .../lib/chain/TestSingleElementChain.java       |  4 ++
 .../lib/db/TestDataDrivenDBInputFormat.java     | 40 ++++++++++++++------
 .../mapreduce/lib/input/TestMultipleInputs.java |  6 ++-
 .../lib/jobcontrol/TestMapReduceJobControl.java |  8 +++-
 .../lib/map/TestMultithreadedMapper.java        | 12 +++++-
 .../lib/output/TestJobOutputCommitter.java      | 16 ++++++--
 .../lib/output/TestMRMultipleOutputs.java       | 17 ++++++++-
 .../TestMRKeyFieldBasedComparator.java          | 20 +++++++---
 .../hadoop/examples/terasort/TestTeraSort.java  | 11 +++++-
 25 files changed, 285 insertions(+), 68 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/ef68b441/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/conf/TestNoDefaultsJobConf.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/conf/TestNoDefaultsJobConf.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/conf/TestNoDefaultsJobConf.java
index b47d5ef..e2d75ab 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/conf/TestNoDefaultsJobConf.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/conf/TestNoDefaultsJobConf.java
@@ -17,17 +17,30 @@
  */
 package org.apache.hadoop.conf;
 
-import org.junit.Assert;
-
-import org.apache.hadoop.mapred.*;
-import org.apache.hadoop.mapreduce.MRConfig;
-import org.apache.hadoop.mapreduce.server.jobtracker.JTConfig;
-import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.FileUtil;
+import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.LongWritable;
 import org.apache.hadoop.io.Text;
-
-import java.io.*;
+import org.apache.hadoop.mapred.FileInputFormat;
+import org.apache.hadoop.mapred.FileOutputFormat;
+import org.apache.hadoop.mapred.HadoopTestCase;
+import org.apache.hadoop.mapred.JobClient;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.TextInputFormat;
+import org.apache.hadoop.mapred.TextOutputFormat;
+import org.apache.hadoop.mapred.Utils;
+import org.junit.Test;
+
+import java.io.BufferedReader;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.io.OutputStream;
+import java.io.OutputStreamWriter;
+import java.io.Writer;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
 
 /**
  * This testcase tests that a JobConf without default values submits jobs
@@ -40,6 +53,7 @@ public class TestNoDefaultsJobConf extends HadoopTestCase {
     super(HadoopTestCase.CLUSTER_MR, HadoopTestCase.DFS_FS, 1, 1);
   }
 
+  @Test
   public void testNoDefaults() throws Exception {
     JobConf configuration = new JobConf();
     assertTrue(configuration.get("hadoop.tmp.dir", null) != null);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ef68b441/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/HadoopTestCase.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/HadoopTestCase.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/HadoopTestCase.java
index 3cd0668..277c0fd 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/HadoopTestCase.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/HadoopTestCase.java
@@ -18,13 +18,12 @@
 
 package org.apache.hadoop.mapred;
 
-import junit.framework.TestCase;
-import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.mapreduce.MRConfig;
+import org.junit.After;
+import org.junit.Before;
 
-import java.io.File;
 import java.io.IOException;
 
 /**
@@ -38,7 +37,7 @@ import java.io.IOException;
  * Job Configurations should be created using a configuration returned by the
  * 'createJobConf()' method.
  */
-public abstract class HadoopTestCase extends TestCase {
+public abstract class HadoopTestCase {
   public static final int LOCAL_MR = 1;
   public static final int CLUSTER_MR = 2;
   public static final int LOCAL_FS = 4;
@@ -140,8 +139,8 @@ public abstract class HadoopTestCase extends TestCase {
    *
    * @throws Exception
    */
-  protected void setUp() throws Exception {
-    super.setUp();
+  @Before
+  public void setUp() throws Exception {
     if (localFS) {
       fileSystem = FileSystem.getLocal(new JobConf());
     }
@@ -164,7 +163,8 @@ public abstract class HadoopTestCase extends TestCase {
    *
    * @throws Exception
    */
-  protected void tearDown() throws Exception {
+  @After
+  public void tearDown() throws Exception {
     try {
       if (mrCluster != null) {
         mrCluster.shutdown();
@@ -181,7 +181,6 @@ public abstract class HadoopTestCase extends TestCase {
     catch (Exception ex) {
       System.out.println(ex);
     }
-    super.tearDown();
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ef68b441/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/NotificationTestCase.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/NotificationTestCase.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/NotificationTestCase.java
index d2ea74e..1f657cf 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/NotificationTestCase.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/NotificationTestCase.java
@@ -34,6 +34,13 @@ import javax.servlet.ServletException;
 import java.io.IOException;
 import java.io.DataOutputStream;
 
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.assertEquals;
+import org.junit.Before;
+import org.junit.After;
+import org.junit.Test;
+
+
 /**
  * Base class to test Job end notification in local and cluster mode.
  *
@@ -140,17 +147,19 @@ public abstract class NotificationTestCase extends HadoopTestCase {
     return conf;
   }
 
-
-  protected void setUp() throws Exception {
+  @Before
+  public void setUp() throws Exception {
     super.setUp();
     startHttpServer();
   }
 
-  protected void tearDown() throws Exception {
+  @After
+  public void tearDown() throws Exception {
     stopHttpServer();
     super.tearDown();
   }
 
+  @Test
   public void testMR() throws Exception {
 
     System.out.println(launchWordCount(this.createJobConf(),

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ef68b441/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestFileOutputFormat.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestFileOutputFormat.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestFileOutputFormat.java
index 81b53cc..3141235 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestFileOutputFormat.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestFileOutputFormat.java
@@ -30,12 +30,16 @@ import java.io.IOException;
 import java.io.OutputStream;
 import java.util.Iterator;
 
+import org.junit.Test;
+import static org.junit.Assert.assertTrue;
+
 public class TestFileOutputFormat extends HadoopTestCase {
 
   public TestFileOutputFormat() throws IOException {
     super(HadoopTestCase.LOCAL_MR, HadoopTestCase.LOCAL_FS, 1, 1);
   }
 
+  @Test
   public void testCustomFile() throws Exception {
     Path inDir = new Path("testing/fileoutputformat/input");
     Path outDir = new Path("testing/fileoutputformat/output");

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ef68b441/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestTaskCommit.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestTaskCommit.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestTaskCommit.java
index bad06e9..bed545e 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestTaskCommit.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestTaskCommit.java
@@ -17,9 +17,6 @@
  */
 package org.apache.hadoop.mapred;
 
-import java.io.File;
-import java.io.IOException;
-
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.Path;
@@ -27,9 +24,18 @@ import org.apache.hadoop.io.Text;
 import org.apache.hadoop.ipc.ProtocolSignature;
 import org.apache.hadoop.mapred.SortedRanges.Range;
 import org.apache.hadoop.mapreduce.TaskType;
-import org.apache.hadoop.mapreduce.checkpoint.CheckpointID;
-import org.apache.hadoop.mapreduce.checkpoint.FSCheckpointID;
 import org.apache.hadoop.mapreduce.checkpoint.TaskCheckpointID;
+import org.junit.After;
+import org.junit.Test;
+
+import java.io.File;
+import java.io.IOException;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
 
 
 public class TestTaskCommit extends HadoopTestCase {
@@ -80,12 +86,13 @@ public class TestTaskCommit extends HadoopTestCase {
     super(LOCAL_MR, LOCAL_FS, 1, 1);
   }
   
-  @Override
+  @After
   public void tearDown() throws Exception {
     super.tearDown();
     FileUtil.fullyDelete(new File(rootDir.toString()));
   }
-  
+
+  @Test
   public void testCommitFail() throws IOException {
     final Path inDir = new Path(rootDir, "./input");
     final Path outDir = new Path(rootDir, "./output");
@@ -199,6 +206,7 @@ public class TestTaskCommit extends HadoopTestCase {
    * 
    * @throws Exception
    */
+  @Test
   public void testTaskCleanupDoesNotCommit() throws Exception {
     // Mimic a job with a special committer that does not cleanup
     // files when a task fails.
@@ -245,23 +253,27 @@ public class TestTaskCommit extends HadoopTestCase {
     assertTrue("Task did not succeed", umbilical.taskDone);
   }
 
+  @Test
   public void testCommitRequiredForMapTask() throws Exception {
     Task testTask = createDummyTask(TaskType.MAP);
     assertTrue("MapTask should need commit", testTask.isCommitRequired());
   }
 
+  @Test
   public void testCommitRequiredForReduceTask() throws Exception {
     Task testTask = createDummyTask(TaskType.REDUCE);
     assertTrue("ReduceTask should need commit", testTask.isCommitRequired());
   }
-  
+
+  @Test
   public void testCommitNotRequiredForJobSetup() throws Exception {
     Task testTask = createDummyTask(TaskType.MAP);
     testTask.setJobSetupTask();
     assertFalse("Job setup task should not need commit", 
         testTask.isCommitRequired());
   }
-  
+
+  @Test
   public void testCommitNotRequiredForJobCleanup() throws Exception {
     Task testTask = createDummyTask(TaskType.MAP);
     testTask.setJobCleanupTask();
@@ -269,6 +281,7 @@ public class TestTaskCommit extends HadoopTestCase {
         testTask.isCommitRequired());
   }
 
+  @Test
   public void testCommitNotRequiredForTaskCleanup() throws Exception {
     Task testTask = createDummyTask(TaskType.REDUCE);
     testTask.setTaskCleanupTask();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ef68b441/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/jobcontrol/TestLocalJobControl.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/jobcontrol/TestLocalJobControl.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/jobcontrol/TestLocalJobControl.java
index 8d35dcf..07b1306 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/jobcontrol/TestLocalJobControl.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/jobcontrol/TestLocalJobControl.java
@@ -27,6 +27,8 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.mapred.HadoopTestCase;
 import org.apache.hadoop.mapred.JobConf;
+import org.junit.Test;
+import static org.junit.Assert.assertEquals;
 
 /**
  * HadoopTestCase that tests the local job runner.
@@ -59,6 +61,7 @@ public class TestLocalJobControl extends HadoopTestCase {
    * object. Finally, it creates a thread to run the JobControl object and
    * monitors/reports the job states.
    */
+  @Test
   public void testLocalJobControlDataCopy() throws Exception {
 
     FileSystem fs = FileSystem.get(createJobConf());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ef68b441/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/lib/TestChainMapReduce.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/lib/TestChainMapReduce.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/lib/TestChainMapReduce.java
index 37cb91f..0933ece 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/lib/TestChainMapReduce.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/lib/TestChainMapReduce.java
@@ -21,12 +21,29 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.LongWritable;
 import org.apache.hadoop.io.Text;
-import org.apache.hadoop.mapred.*;
+import org.apache.hadoop.mapred.FileInputFormat;
+import org.apache.hadoop.mapred.FileOutputFormat;
+import org.apache.hadoop.mapred.HadoopTestCase;
+import org.apache.hadoop.mapred.JobClient;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.Mapper;
+import org.apache.hadoop.mapred.OutputCollector;
+import org.apache.hadoop.mapred.Reducer;
+import org.apache.hadoop.mapred.Reporter;
+import org.apache.hadoop.mapred.RunningJob;
+import org.apache.hadoop.mapred.TextInputFormat;
+import org.apache.hadoop.mapred.TextOutputFormat;
+import org.junit.Test;
 
 import java.io.DataOutputStream;
 import java.io.IOException;
 import java.util.Iterator;
 
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotSame;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
 public class TestChainMapReduce extends HadoopTestCase {
 
   private static Path getFlagDir(boolean local) {
@@ -67,6 +84,7 @@ public class TestChainMapReduce extends HadoopTestCase {
     super(HadoopTestCase.LOCAL_MR, HadoopTestCase.LOCAL_FS, 1, 1);
   }
 
+  @Test
   public void testChain() throws Exception {
     Path inDir = new Path("testing/chain/input");
     Path outDir = new Path("testing/chain/output");

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ef68b441/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/lib/TestKeyFieldBasedComparator.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/lib/TestKeyFieldBasedComparator.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/lib/TestKeyFieldBasedComparator.java
index 34a4d2c..35b3f24 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/lib/TestKeyFieldBasedComparator.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/lib/TestKeyFieldBasedComparator.java
@@ -36,6 +36,8 @@ import org.apache.hadoop.mapred.TextOutputFormat;
 import org.apache.hadoop.mapred.Utils;
 import org.junit.After;
 import org.junit.Test;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
 
 import java.io.BufferedReader;
 import java.io.File;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ef68b441/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/lib/TestMultipleInputs.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/lib/TestMultipleInputs.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/lib/TestMultipleInputs.java
index e5c6d75..3a9cb9e 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/lib/TestMultipleInputs.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/lib/TestMultipleInputs.java
@@ -17,11 +17,6 @@
  */
 package org.apache.hadoop.mapred.lib;
 
-import java.io.IOException;
-import java.util.Map;
-
-import junit.framework.TestCase;
-
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.mapred.InputFormat;
 import org.apache.hadoop.mapred.JobConf;
@@ -30,12 +25,19 @@ import org.apache.hadoop.mapred.Mapper;
 import org.apache.hadoop.mapred.OutputCollector;
 import org.apache.hadoop.mapred.Reporter;
 import org.apache.hadoop.mapred.TextInputFormat;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.util.Map;
+
+import static org.junit.Assert.assertEquals;
 
 /**
  * @see TestDelegatingInputFormat
  */
-public class TestMultipleInputs extends TestCase {
-  
+public class TestMultipleInputs {
+
+  @Test
   public void testAddInputPathWithFormat() {
     final JobConf conf = new JobConf();
     MultipleInputs.addInputPath(conf, new Path("/foo"), TextInputFormat.class);
@@ -48,6 +50,7 @@ public class TestMultipleInputs extends TestCase {
        .getClass());
   }
 
+  @Test
   public void testAddInputPathWithMapper() {
     final JobConf conf = new JobConf();
     MultipleInputs.addInputPath(conf, new Path("/foo"), TextInputFormat.class,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ef68b441/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/lib/TestMultipleOutputs.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/lib/TestMultipleOutputs.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/lib/TestMultipleOutputs.java
index 59c0a97..f3e5893 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/lib/TestMultipleOutputs.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/lib/TestMultipleOutputs.java
@@ -24,7 +24,23 @@ import org.apache.hadoop.io.LongWritable;
 import org.apache.hadoop.io.SequenceFile;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.serializer.JavaSerializationComparator;
-import org.apache.hadoop.mapred.*;
+import org.apache.hadoop.mapred.Counters;
+import org.apache.hadoop.mapred.FileInputFormat;
+import org.apache.hadoop.mapred.FileOutputFormat;
+import org.apache.hadoop.mapred.HadoopTestCase;
+import org.apache.hadoop.mapred.JobClient;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.Mapper;
+import org.apache.hadoop.mapred.OutputCollector;
+import org.apache.hadoop.mapred.Reducer;
+import org.apache.hadoop.mapred.Reporter;
+import org.apache.hadoop.mapred.RunningJob;
+import org.apache.hadoop.mapred.SequenceFileOutputFormat;
+import org.apache.hadoop.mapred.TextInputFormat;
+import org.apache.hadoop.mapred.TextOutputFormat;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
 
 import java.io.BufferedReader;
 import java.io.DataOutputStream;
@@ -32,17 +48,23 @@ import java.io.IOException;
 import java.io.InputStreamReader;
 import java.util.Iterator;
 
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
 public class TestMultipleOutputs extends HadoopTestCase {
 
   public TestMultipleOutputs() throws IOException {
     super(HadoopTestCase.LOCAL_MR, HadoopTestCase.LOCAL_FS, 1, 1);
   }
 
+  @Test
   public void testWithoutCounters() throws Exception {
     _testMultipleOutputs(false);
     _testMOWithJavaSerialization(false);
   }
 
+  @Test
   public void testWithCounters() throws Exception {
     _testMultipleOutputs(true);
     _testMOWithJavaSerialization(true);
@@ -62,6 +84,7 @@ public class TestMultipleOutputs extends HadoopTestCase {
     return dir;
   }
 
+  @Before
   public void setUp() throws Exception {
     super.setUp();
     Path rootDir = getDir(ROOT_DIR);
@@ -75,6 +98,7 @@ public class TestMultipleOutputs extends HadoopTestCase {
     }
   }
 
+  @After
   public void tearDown() throws Exception {
     Path rootDir = getDir(ROOT_DIR);
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ef68b441/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/lib/TestMultithreadedMapRunner.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/lib/TestMultithreadedMapRunner.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/lib/TestMultithreadedMapRunner.java
index 7e224cd..1059d29 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/lib/TestMultithreadedMapRunner.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/lib/TestMultithreadedMapRunner.java
@@ -22,26 +22,45 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.LongWritable;
 import org.apache.hadoop.io.Text;
-import org.apache.hadoop.mapred.*;
+import org.apache.hadoop.mapred.FileInputFormat;
+import org.apache.hadoop.mapred.FileOutputFormat;
+import org.apache.hadoop.mapred.HadoopTestCase;
+import org.apache.hadoop.mapred.JobClient;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.Mapper;
+import org.apache.hadoop.mapred.OutputCollector;
+import org.apache.hadoop.mapred.Reducer;
+import org.apache.hadoop.mapred.Reporter;
+import org.apache.hadoop.mapred.RunningJob;
+import org.apache.hadoop.mapred.TextInputFormat;
+import org.apache.hadoop.mapred.TextOutputFormat;
 import org.apache.hadoop.mapreduce.lib.map.MultithreadedMapper;
+import org.junit.Test;
 
 import java.io.DataOutputStream;
 import java.io.IOException;
 import java.util.Iterator;
 
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
 public class TestMultithreadedMapRunner extends HadoopTestCase {
 
   public TestMultithreadedMapRunner() throws IOException {
     super(HadoopTestCase.LOCAL_MR, HadoopTestCase.LOCAL_FS, 1, 1);
   }
 
+  @Test
   public void testOKRun() throws Exception {
     run(false, false);
   }
 
+  @Test
   public void testIOExRun() throws Exception {
     run(true, false);
   }
+
+  @Test
   public void testRuntimeExRun() throws Exception {
     run(false, true);
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ef68b441/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestChild.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestChild.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestChild.java
index d5afe63..338f117 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestChild.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestChild.java
@@ -30,6 +30,13 @@ import org.apache.hadoop.io.Text;
 import org.apache.hadoop.mapred.HadoopTestCase;
 import org.apache.hadoop.mapred.JobConf;
 import org.apache.log4j.Level;
+import org.junit.Before;
+import org.junit.Test;
+
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
 
 public class TestChild extends HadoopTestCase {
   private static String TEST_ROOT_DIR =
@@ -145,7 +152,8 @@ public class TestChild extends HadoopTestCase {
         + numPartFiles, list.length == numPartFiles);
     return job;
   }
-  
+
+  @Test
   public void testChild() throws Exception {
     try {
       submitAndValidateJob(createJobConf(), 1, 1, true);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ef68b441/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestNoJobSetupCleanup.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestNoJobSetupCleanup.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestNoJobSetupCleanup.java
index 5d36c92..7520f38 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestNoJobSetupCleanup.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestNoJobSetupCleanup.java
@@ -30,6 +30,9 @@ import org.apache.hadoop.mapred.HadoopTestCase;
 import org.apache.hadoop.mapreduce.lib.output.TextOutputFormat;
 import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl;
 import org.junit.Ignore;
+import org.junit.Test;
+import static org.junit.Assert.assertTrue;
+
 @Ignore
 public class TestNoJobSetupCleanup extends HadoopTestCase {
   private static String TEST_ROOT_DIR =
@@ -68,7 +71,8 @@ public class TestNoJobSetupCleanup extends HadoopTestCase {
         + numPartFiles, list.length == numPartFiles);
     return job;
   }
-  
+
+  @Test
   public void testNoJobSetupCleanup() throws Exception {
     try {
       Configuration conf = createJobConf();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ef68b441/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestTaskContext.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestTaskContext.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestTaskContext.java
index bf742c4..67daaa4 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestTaskContext.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestTaskContext.java
@@ -33,6 +33,8 @@ import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.Ignore;
 import org.junit.Test;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
 
 /**
  * Tests context api and {@link StatusReporter#getProgress()} via 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ef68b441/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/chain/TestChainErrors.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/chain/TestChainErrors.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/chain/TestChainErrors.java
index 2dfcf41..46024bc 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/chain/TestChainErrors.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/chain/TestChainErrors.java
@@ -29,6 +29,10 @@ import org.apache.hadoop.mapreduce.Job;
 import org.apache.hadoop.mapreduce.MapReduceTestUtil;
 import org.apache.hadoop.mapreduce.Mapper;
 import org.apache.hadoop.mapreduce.Reducer;
+import org.junit.Test;
+
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.assertEquals;
 
 /**
  * Tests error conditions in ChainMapper/ChainReducer.
@@ -51,6 +55,7 @@ public class TestChainErrors extends HadoopTestCase {
    * 
    * @throws Exception
    */
+  @Test
   public void testChainSubmission() throws Exception {
 
     Configuration conf = createJobConf();
@@ -89,6 +94,7 @@ public class TestChainErrors extends HadoopTestCase {
    * 
    * @throws Exception
    */
+  @Test
   public void testChainFail() throws Exception {
 
     Configuration conf = createJobConf();
@@ -114,6 +120,7 @@ public class TestChainErrors extends HadoopTestCase {
    * 
    * @throws Exception
    */
+  @Test
   public void testReducerFail() throws Exception {
 
     Configuration conf = createJobConf();
@@ -139,6 +146,7 @@ public class TestChainErrors extends HadoopTestCase {
    * 
    * @throws Exception
    */
+  @Test
   public void testChainMapNoOuptut() throws Exception {
     Configuration conf = createJobConf();
     String expectedOutput = "";
@@ -163,6 +171,7 @@ public class TestChainErrors extends HadoopTestCase {
    * 
    * @throws Exception
    */
+  @Test
   public void testChainReduceNoOuptut() throws Exception {
     Configuration conf = createJobConf();
     String expectedOutput = "";

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ef68b441/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/chain/TestMapReduceChain.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/chain/TestMapReduceChain.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/chain/TestMapReduceChain.java
index 971ea68..aaaaf51 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/chain/TestMapReduceChain.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/chain/TestMapReduceChain.java
@@ -30,6 +30,11 @@ import org.apache.hadoop.mapreduce.Job;
 import org.apache.hadoop.mapreduce.MapReduceTestUtil;
 import org.apache.hadoop.mapreduce.Mapper;
 import org.apache.hadoop.mapreduce.Reducer;
+import org.junit.Test;
+
+import static org.junit.Assert.fail;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
 
 public class TestMapReduceChain extends HadoopTestCase {
 
@@ -63,6 +68,7 @@ public class TestMapReduceChain extends HadoopTestCase {
     super(HadoopTestCase.LOCAL_MR, HadoopTestCase.LOCAL_FS, 1, 1);
   }
 
+  @Test
   public void testChain() throws Exception {
     Path inDir = new Path(localPathRoot, "testing/chain/input");
     Path outDir = new Path(localPathRoot, "testing/chain/output");

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ef68b441/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/chain/TestSingleElementChain.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/chain/TestSingleElementChain.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/chain/TestSingleElementChain.java
index 06cfe1c..f78ac70 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/chain/TestSingleElementChain.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/chain/TestSingleElementChain.java
@@ -26,6 +26,9 @@ import org.apache.hadoop.mapreduce.Job;
 import org.apache.hadoop.mapreduce.MapReduceTestUtil;
 import org.apache.hadoop.mapreduce.lib.map.TokenCounterMapper;
 import org.apache.hadoop.mapreduce.lib.reduce.IntSumReducer;
+import org.junit.Test;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
 
 import java.io.IOException;
 
@@ -42,6 +45,7 @@ public class TestSingleElementChain extends HadoopTestCase {
   }
 
   // test chain mapper and reducer by adding single mapper and reducer to chain
+  @Test
   public void testNoChain() throws Exception {
     Path inDir = new Path(localPathRoot, "testing/chain/input");
     Path outDir = new Path(localPathRoot, "testing/chain/output");

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ef68b441/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/db/TestDataDrivenDBInputFormat.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/db/TestDataDrivenDBInputFormat.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/db/TestDataDrivenDBInputFormat.java
index 37f9364..81a3249 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/db/TestDataDrivenDBInputFormat.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/db/TestDataDrivenDBInputFormat.java
@@ -18,25 +18,40 @@
 
 package org.apache.hadoop.mapreduce.lib.db;
 
-import java.sql.*;
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
-//import org.apache.hadoop.examples.DBCountPageView;
-import org.apache.hadoop.fs.*;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.NullWritable;
 import org.apache.hadoop.io.WritableComparable;
 import org.apache.hadoop.mapred.HadoopTestCase;
-import org.apache.hadoop.mapreduce.*;
-import org.apache.hadoop.mapreduce.lib.db.*;
-import org.apache.hadoop.mapreduce.lib.input.*;
-import org.apache.hadoop.mapreduce.lib.output.*;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.Mapper;
+import org.apache.hadoop.mapreduce.Reducer;
+import org.apache.hadoop.mapreduce.TaskCounter;
+import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
 import org.apache.hadoop.util.StringUtils;
 import org.hsqldb.server.Server;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.sql.Connection;
+import java.sql.Date;
+import java.sql.DriverManager;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.sql.Statement;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+//import org.apache.hadoop.examples.DBCountPageView;
 
 /**
  * Test aspects of DataDrivenDBInputFormat
@@ -109,11 +124,13 @@ public class TestDataDrivenDBInputFormat extends HadoopTestCase {
     createConnection(driverClassName, url);
   }
 
+  @Before
   public void setUp() throws Exception {
     initialize(DRIVER_CLASS, DB_URL);
     super.setUp();
   }
 
+  @After
   public void tearDown() throws Exception {
     super.tearDown();
     shutdown();
@@ -170,6 +187,7 @@ public class TestDataDrivenDBInputFormat extends HadoopTestCase {
     }
   }
 
+  @Test
   public void testDateSplits() throws Exception {
     Statement s = connection.createStatement();
     final String DATE_TABLE = "datetable";

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ef68b441/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/input/TestMultipleInputs.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/input/TestMultipleInputs.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/input/TestMultipleInputs.java
index c868050..632c40e 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/input/TestMultipleInputs.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/input/TestMultipleInputs.java
@@ -38,6 +38,8 @@ import org.apache.hadoop.mapreduce.Mapper;
 import org.apache.hadoop.mapreduce.Reducer;
 import org.junit.Before;
 import org.junit.Test;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
 
 /**
  * @see TestDelegatingInputFormat
@@ -139,7 +141,7 @@ public class TestMultipleInputs extends HadoopTestCase {
     assertTrue(output.readLine().equals("e 2"));
   }
 
-  @SuppressWarnings("unchecked")
+  @Test
   public void testAddInputPathWithFormat() throws IOException {
     final Job conf = Job.getInstance();
     MultipleInputs.addInputPath(conf, new Path("/foo"), TextInputFormat.class);
@@ -152,7 +154,7 @@ public class TestMultipleInputs extends HadoopTestCase {
        .getClass());
   }
 
-  @SuppressWarnings("unchecked")
+  @Test
   public void testAddInputPathWithMapper() throws IOException {
     final Job conf = Job.getInstance();
     MultipleInputs.addInputPath(conf, new Path("/foo"), TextInputFormat.class,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ef68b441/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/jobcontrol/TestMapReduceJobControl.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/jobcontrol/TestMapReduceJobControl.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/jobcontrol/TestMapReduceJobControl.java
index 14c64bd..d86ddd0 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/jobcontrol/TestMapReduceJobControl.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/jobcontrol/TestMapReduceJobControl.java
@@ -33,6 +33,9 @@ import org.apache.hadoop.mapred.HadoopTestCase;
 import org.apache.hadoop.mapreduce.Job;
 import org.apache.hadoop.mapreduce.MapReduceTestUtil;
 import org.junit.Test;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
 
 /**
  * This class performs unit test for Job/JobControl classes.
@@ -120,7 +123,8 @@ public class TestMapReduceJobControl extends HadoopTestCase {
       } catch (Exception e) {}
     }
   }
-  
+
+  @Test
   public void testJobControlWithFailJob() throws Exception {
     LOG.info("Starting testJobControlWithFailJob");
     Configuration conf = createJobConf();
@@ -144,6 +148,7 @@ public class TestMapReduceJobControl extends HadoopTestCase {
     theControl.stop();
   }
 
+  @Test
   public void testJobControlWithKillJob() throws Exception {
     LOG.info("Starting testJobControlWithKillJob");
 
@@ -182,6 +187,7 @@ public class TestMapReduceJobControl extends HadoopTestCase {
     theControl.stop();
   }
 
+  @Test
   public void testJobControl() throws Exception {
     LOG.info("Starting testJobControl");
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ef68b441/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/map/TestMultithreadedMapper.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/map/TestMultithreadedMapper.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/map/TestMultithreadedMapper.java
index 52b0e70..5096192 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/map/TestMultithreadedMapper.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/map/TestMultithreadedMapper.java
@@ -23,23 +23,33 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.LongWritable;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.mapred.HadoopTestCase;
-import org.apache.hadoop.mapreduce.*;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.MapReduceTestUtil;
+import org.apache.hadoop.mapreduce.Mapper;
+import org.apache.hadoop.mapreduce.Reducer;
+import org.junit.Test;
 
 import java.io.IOException;
 
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
 public class TestMultithreadedMapper extends HadoopTestCase {
 
   public TestMultithreadedMapper() throws IOException {
     super(HadoopTestCase.LOCAL_MR, HadoopTestCase.LOCAL_FS, 1, 1);
   }
 
+  @Test
   public void testOKRun() throws Exception {
     run(false, false);
   }
 
+  @Test
   public void testIOExRun() throws Exception {
     run(true, false);
   }
+  @Test
   public void testRuntimeExRun() throws Exception {
     run(false, true);
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ef68b441/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/output/TestJobOutputCommitter.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/output/TestJobOutputCommitter.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/output/TestJobOutputCommitter.java
index 49b59ca..19b712f 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/output/TestJobOutputCommitter.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/output/TestJobOutputCommitter.java
@@ -33,6 +33,11 @@ import org.apache.hadoop.mapreduce.MapReduceTestUtil;
 import org.apache.hadoop.mapreduce.OutputCommitter;
 import org.apache.hadoop.mapreduce.OutputFormat;
 import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.assertFalse;
 
 /**
  * A JUnit test to test Map-Reduce job committer.
@@ -54,15 +59,15 @@ public class TestJobOutputCommitter extends HadoopTestCase {
   private FileSystem fs;
   private Configuration conf = null;
 
-  @Override
-  protected void setUp() throws Exception {
+  @Before
+  public void setUp() throws Exception {
     super.setUp();
     conf = createJobConf();
     fs = getFileSystem();
   }
 
-  @Override
-  protected void tearDown() throws Exception {
+  @After
+  public void tearDown() throws Exception {
     fs.delete(new Path(TEST_ROOT_DIR), true);
     super.tearDown();
   }
@@ -219,6 +224,7 @@ public class TestJobOutputCommitter extends HadoopTestCase {
    * 
    * @throws Exception
    */
+  @Test
   public void testDefaultCleanupAndAbort() throws Exception {
     // check with a successful job
     testSuccessfulJob(FileOutputCommitter.SUCCEEDED_FILE_NAME,
@@ -238,6 +244,7 @@ public class TestJobOutputCommitter extends HadoopTestCase {
    * 
    * @throws Exception
    */
+  @Test
   public void testCustomAbort() throws Exception {
     // check with a successful job
     testSuccessfulJob(FileOutputCommitter.SUCCEEDED_FILE_NAME,
@@ -264,6 +271,7 @@ public class TestJobOutputCommitter extends HadoopTestCase {
    * compatibility testing.
    * @throws Exception 
    */
+  @Test
   public void testCustomCleanup() throws Exception {
     // check with a successful job
     testSuccessfulJob(CUSTOM_CLEANUP_FILE_NAME, 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ef68b441/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/output/TestMRMultipleOutputs.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/output/TestMRMultipleOutputs.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/output/TestMRMultipleOutputs.java
index 6c432dd..babd20e 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/output/TestMRMultipleOutputs.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/output/TestMRMultipleOutputs.java
@@ -27,23 +27,36 @@ import org.apache.hadoop.io.SequenceFile;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.serializer.JavaSerializationComparator;
 import org.apache.hadoop.mapred.HadoopTestCase;
-import org.apache.hadoop.mapreduce.*;
+import org.apache.hadoop.mapreduce.CounterGroup;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.MapReduceTestUtil;
+import org.apache.hadoop.mapreduce.Mapper;
+import org.apache.hadoop.mapreduce.Reducer;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
 
 import java.io.BufferedReader;
 import java.io.IOException;
 import java.io.InputStreamReader;
 
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
 public class TestMRMultipleOutputs extends HadoopTestCase {
 
   public TestMRMultipleOutputs() throws IOException {
     super(HadoopTestCase.LOCAL_MR, HadoopTestCase.LOCAL_FS, 1, 1);
   }
 
+  @Test
   public void testWithoutCounters() throws Exception {
     _testMultipleOutputs(false);
     _testMOWithJavaSerialization(false);
   }
 
+  @Test
   public void testWithCounters() throws Exception {
     _testMultipleOutputs(true);
     _testMOWithJavaSerialization(true);
@@ -57,6 +70,7 @@ public class TestMRMultipleOutputs extends HadoopTestCase {
   private static String TEXT = "text";
   private static String SEQUENCE = "sequence";
 
+  @Before
   public void setUp() throws Exception {
     super.setUp();
     Configuration conf = createJobConf();
@@ -64,6 +78,7 @@ public class TestMRMultipleOutputs extends HadoopTestCase {
     fs.delete(ROOT_DIR, true);
   }
 
+  @After
   public void tearDown() throws Exception {
     Configuration conf = createJobConf();
     FileSystem fs = FileSystem.get(conf);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ef68b441/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/partition/TestMRKeyFieldBasedComparator.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/partition/TestMRKeyFieldBasedComparator.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/partition/TestMRKeyFieldBasedComparator.java
index 3a2b831..0d75d2f 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/partition/TestMRKeyFieldBasedComparator.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/partition/TestMRKeyFieldBasedComparator.java
@@ -18,8 +18,6 @@
 
 package org.apache.hadoop.mapreduce.lib.partition;
 
-import java.io.*;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.Path;
@@ -32,6 +30,15 @@ import org.apache.hadoop.mapreduce.MRJobConfig;
 import org.apache.hadoop.mapreduce.MapReduceTestUtil;
 import org.apache.hadoop.mapreduce.Reducer;
 import org.apache.hadoop.mapreduce.lib.map.InverseMapper;
+import org.junit.Test;
+
+import java.io.BufferedReader;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
 
 
 public class TestMRKeyFieldBasedComparator extends HadoopTestCase {
@@ -45,8 +52,8 @@ public class TestMRKeyFieldBasedComparator extends HadoopTestCase {
     conf = createJobConf();
     conf.set(MRJobConfig.MAP_OUTPUT_KEY_FIELD_SEPERATOR, " ");
   }
-  
-  private void testComparator(String keySpec, int expect) 
+
+  private void testComparator(String keySpec, int expect)
       throws Exception {
     String root = System.getProperty("test.build.data", "/tmp");
     Path inDir = new Path(root, "test_cmp/in");
@@ -93,7 +100,8 @@ public class TestMRKeyFieldBasedComparator extends HadoopTestCase {
       reader.close();
     }
   }
-  
+
+  @Test
   public void testBasicUnixComparator() throws Exception {
     testComparator("-k1,1n", 1);
     testComparator("-k2,2n", 1);
@@ -117,7 +125,7 @@ public class TestMRKeyFieldBasedComparator extends HadoopTestCase {
   byte[] line1_bytes = line1.getBytes();
   byte[] line2_bytes = line2.getBytes();
 
-  public void testWithoutMRJob(String keySpec, int expect) throws Exception {
+  private void testWithoutMRJob(String keySpec, int expect) throws Exception {
     KeyFieldBasedComparator<Void, Void> keyFieldCmp = 
       new KeyFieldBasedComparator<Void, Void>();
     conf.set("mapreduce.partition.keycomparator.options", keySpec);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ef68b441/hadoop-mapreduce-project/hadoop-mapreduce-examples/src/test/java/org/apache/hadoop/examples/terasort/TestTeraSort.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-examples/src/test/java/org/apache/hadoop/examples/terasort/TestTeraSort.java b/hadoop-mapreduce-project/hadoop-mapreduce-examples/src/test/java/org/apache/hadoop/examples/terasort/TestTeraSort.java
index 3492089..391e482 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-examples/src/test/java/org/apache/hadoop/examples/terasort/TestTeraSort.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-examples/src/test/java/org/apache/hadoop/examples/terasort/TestTeraSort.java
@@ -27,6 +27,12 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.mapred.FileAlreadyExistsException;
 import org.apache.hadoop.mapred.HadoopTestCase;
 import org.apache.hadoop.util.ToolRunner;
+import org.junit.After;
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.fail;
+
 public class TestTeraSort extends HadoopTestCase {
   private static Log LOG = LogFactory.getLog(TestTeraSort.class);
   
@@ -35,7 +41,8 @@ public class TestTeraSort extends HadoopTestCase {
     super(LOCAL_MR, LOCAL_FS, 1, 1);
   }
 
-  protected void tearDown() throws Exception {
+  @After
+  public void tearDown() throws Exception {
     getFileSystem().delete(new Path(TEST_DIR), true);
     super.tearDown();
   }
@@ -76,6 +83,7 @@ public class TestTeraSort extends HadoopTestCase {
     assertEquals(ToolRunner.run(job, new TeraValidate(), svArgs), 0);
   }
 
+  @Test
   public void testTeraSort() throws Exception {
     // Run TeraGen to generate input for 'terasort'
     runTeraGen(createJobConf(), SORT_INPUT_PATH);
@@ -104,6 +112,7 @@ public class TestTeraSort extends HadoopTestCase {
       TERA_OUTPUT_PATH);
   }
 
+  @Test
   public void testTeraSortWithLessThanTwoArgs() throws Exception {
     String[] args = new String[1];
     assertEquals(new TeraSort().run(args), 2);


[17/46] hadoop git commit: HDFS-9928. Make HDFS commands guide up to date (Wei-Chiu Chuang via iwasakims)

Posted by ae...@apache.org.
HDFS-9928. Make HDFS commands guide up to date (Wei-Chiu Chuang via iwasakims)


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

Branch: refs/heads/HDFS-7240
Commit: 5de848cd5d46527a8fba481c76089da21f533050
Parents: eba66a6
Author: Masatake Iwasaki <iw...@apache.org>
Authored: Tue Mar 15 20:42:05 2016 +0900
Committer: Masatake Iwasaki <iw...@apache.org>
Committed: Tue Mar 15 20:42:05 2016 +0900

----------------------------------------------------------------------
 .../src/site/markdown/HDFSCommands.md           | 159 ++++++++++++-------
 1 file changed, 103 insertions(+), 56 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/5de848cd/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSCommands.md
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSCommands.md b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSCommands.md
index ae126a2..035abd6 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSCommands.md
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSCommands.md
@@ -19,6 +19,7 @@ HDFS Commands Guide
 * [User Commands](#User_Commands)
     * [classpath](#classpath)
     * [dfs](#dfs)
+    * [envvars](#envvars)
     * [fetchdt](#fetchdt)
     * [fsck](#fsck)
     * [getconf](#getconf)
@@ -88,6 +89,12 @@ Usage: `hdfs dfs [COMMAND [COMMAND_OPTIONS]]`
 
 Run a filesystem command on the file system supported in Hadoop. The various COMMAND\_OPTIONS can be found at [File System Shell Guide](../hadoop-common/FileSystemShell.html).
 
+### `envvars`
+
+Usage: `hdfs envvars`
+
+display computed Hadoop environment variables.
+
 ### `fetchdt`
 
 Usage: `hdfs fetchdt <opts> <token_file_path> `
@@ -182,7 +189,8 @@ Usage: `hdfs jmxget [-localVM ConnectorURL | -port port | -server mbeanserver |
 | `-help` | print help |
 | `-localVM` ConnectorURL | connect to the VM on the same machine |
 | `-port` *mbean server port* | specify mbean server port, if missing it will try to connect to MBean Server in the same VM |
-| `-service` | specify jmx service, either DataNode or NameNode, the default |
+| `-server` | specify mbean server (localhost by default) |
+| `-service` NameNode\|DataNode | specify jmx service. NameNode by default. |
 
 Dump JMX information from a service.
 
@@ -203,11 +211,11 @@ Usage: `hdfs oev [OPTIONS] -i INPUT_FILE -o OUTPUT_FILE`
 |:---- |:---- |
 | `-f`,`--fix-txids` | Renumber the transaction IDs in the input, so that there are no gaps or invalid transaction IDs. |
 | `-h`,`--help` | Display usage information and exit |
-| `-r`,`--ecover` | When reading binary edit logs, use recovery mode. This will give you the chance to skip corrupt parts of the edit log. |
+| `-r`,`--recover` | When reading binary edit logs, use recovery mode. This will give you the chance to skip corrupt parts of the edit log. |
 | `-p`,`--processor` *arg* | Select which type of processor to apply against image file, currently supported processors are: binary (native binary format that Hadoop uses), xml (default, XML format), stats (prints statistics about edits file) |
 | `-v`,`--verbose` | More verbose output, prints the input and output filenames, for processors that write to a file, also output to screen. On large image files this will dramatically increase processing time (default is false). |
 
-Hadoop offline edits viewer.
+Hadoop offline edits viewer. See [Offline Edits Viewer Guide](./HdfsEditsViewer.html) for more info.
 
 ### `oiv`
 
@@ -217,17 +225,24 @@ Usage: `hdfs oiv [OPTIONS] -i INPUT_FILE`
 
 | COMMAND\_OPTION | Description |
 |:---- |:---- |
-| `-i`,`--inputFile` *arg* | edits file to process, xml (case insensitive) extension means XML format, any other filename means binary format |
+| `-i`\|`--inputFile` *input file* | Specify the input fsimage file (or XML file, if ReverseXML processor is used) to process. |
+
 
 #### Optional command line arguments:
 
 | COMMAND\_OPTION | Description |
 |:---- |:---- |
-| `-h`,`--help` | Display usage information and exit |
-| `-o`,`--outputFile` *arg* | Name of output file. If the specified file exists, it will be overwritten, format of the file is determined by -p option |
-| `-p`,`--processor` *arg* | Select which type of processor to apply against image file, currently supported processors are: binary (native binary format that Hadoop uses), xml (default, XML format), stats (prints statistics about edits file) |
+| `-o`,`--outputFile` *output file* | Specify the output filename, if the specified output processor generates one. If the specified file already exists, it is silently overwritten. (output to stdout by default) If the input file is an XML file, it also creates an &lt;outputFile&gt;.md5. |
+| `-p`,`--processor` *processor* | Specify the image processor to apply against the image file. Currently valid options are `Web` (default), `XML`, `Delimited`, `FileDistribution` and `ReverseXML`. |
+| `-addr` *address* | Specify the address(host:port) to listen. (localhost:5978 by default). This option is used with Web processor. |
+| `-maxSize` *size* | Specify the range [0, maxSize] of file sizes to be analyzed in bytes (128GB by default). This option is used with FileDistribution processor. |
+| `-step` *size* | Specify the granularity of the distribution in bytes (2MB by default). This option is used with FileDistribution processor. |
+| `-delimiter` *arg* | Delimiting string to use with Delimited processor. |
+| `-t`,`--temp` *temporary dir* | Use temporary dir to cache intermediate result to generate Delimited outputs. If not set, Delimited processor constructs the namespace in memory before outputting text. |
+| `-h`,`--help` | Display the tool usage and help information and exit. |
 
-Hadoop Offline Image Viewer for newer image files.
+
+Hadoop Offline Image Viewer for image files in Hadoop 2.4 or up. See [Offline Image Viewer Guide](./HdfsImageViewer.html) for more info.
 
 ### `oiv_legacy`
 
@@ -235,11 +250,21 @@ Usage: `hdfs oiv_legacy [OPTIONS] -i INPUT_FILE -o OUTPUT_FILE`
 
 | COMMAND\_OPTION | Description |
 |:---- |:---- |
-| `-h`,`--help` | Display usage information and exit |
-| `-i`,`--inputFile` *arg* | edits file to process, xml (case insensitive) extension means XML format, any other filename means binary format |
-| `-o`,`--outputFile` *arg* | Name of output file. If the specified file exists, it will be overwritten, format of the file is determined by -p option |
+| `-i`,`--inputFile` *input file* | Specify the input fsimage file to process. |
+| `-o`,`--outputFile` *output file* | Specify the output filename, if the specified output processor generates one. If the specified file already exists, it is silently overwritten. |
 
-Hadoop offline image viewer for older versions of Hadoop.
+#### Optional command line arguments:
+
+| COMMAND\_OPTION | Description |
+|:---- |:---- |
+| `-p`\|`--processor` *processor* | Specify the image processor to apply against the image file. Valid options are Ls (default), XML, Delimited, Indented, and FileDistribution. |
+| `-skipBlocks` | Do not enumerate individual blocks within files. This may save processing time and outfile file space on namespaces with very large files. The Ls processor reads the blocks to correctly determine file sizes and ignores this option. |
+| `-printToScreen` | Pipe output of processor to console as well as specified file. On extremely large namespaces, this may increase processing time by an order of magnitude. |
+| `-delimiter` *arg* | When used in conjunction with the Delimited processor, replaces the default tab delimiter with the string specified by *arg*. |
+| `-h`\|`--help` | Display the tool usage and help information and exit. |
+
+
+Hadoop offline image viewer for older versions of Hadoop. See [oiv\_legacy Command](./HdfsImageViewer.html#oiv_legacy_Command) for more info.
 
 ### `snapshotDiff`
 
@@ -263,13 +288,14 @@ Commands useful for administrators of a hadoop cluster.
 Usage:
 
         hdfs balancer
-              [-threshold <threshold>]
               [-policy <policy>]
+              [-threshold <threshold>]
               [-exclude [-f <hosts-file> | <comma-separated list of hosts>]]
               [-include [-f <hosts-file> | <comma-separated list of hosts>]]
               [-source [-f <hosts-file> | <comma-separated list of hosts>]]
               [-blockpools <comma-separated list of blockpool ids>]
               [-idleiterations <idleiterations>]
+              [-runDuringUpgrade]
 
 | COMMAND\_OPTION | Description |
 |:---- |:---- |
@@ -280,6 +306,8 @@ Usage:
 | `-source -f` \<hosts-file\> \| \<comma-separated list of hosts\> | Pick only the specified datanodes as source nodes. |
 | `-blockpools` \<comma-separated list of blockpool ids\> | The balancer will only run on blockpools included in this list. |
 | `-idleiterations` \<iterations\> | Maximum number of idle iterations before exit. This overwrites the default idleiterations(5). |
+| `-runDuringUpgrade` | Whether to run the balancer during an ongoing HDFS upgrade. This is usually not desired since it will not affect used space on over-utilized machines. |
+| `-h`\|`--help` | Display the tool usage and help information and exit. |
 
 Runs a cluster balancing utility. An administrator can simply press Ctrl-C to stop the rebalancing process. See [Balancer](./HdfsUserGuide.html#Balancer) for more details.
 
@@ -289,7 +317,18 @@ Besides the above command options, a pinning feature is introduced starting from
 
 ### `cacheadmin`
 
-Usage: `hdfs cacheadmin -addDirective -path <path> -pool <pool-name> [-force] [-replication <replication>] [-ttl <time-to-live>]`
+Usage:
+
+    hdfs cacheadmin [-addDirective -path <path> -pool <pool-name> [-force] [-replication <replication>] [-ttl <time-to-live>]]
+    hdfs cacheadmin [-modifyDirective -id <id> [-path <path>] [-force] [-replication <replication>] [-pool <pool-name>] [-ttl <time-to-live>]]
+    hdfs cacheadmin [-listDirectives [-stats] [-path <path>] [-pool <pool>] [-id <id>]
+    hdfs cacheadmin [-removeDirective <id>]
+    hdfs cacheadmin [-removeDirectives -path <path>]
+    hdfs cacheadmin [-addPool <name> [-owner <owner>] [-group <group>] [-mode <mode>] [-limit <limit>] [-maxTtl <maxTtl>]
+    hdfs cacheadmin [-modifyPool <name> [-owner <owner>] [-group <group>] [-mode <mode>] [-limit <limit>] [-maxTtl <maxTtl>]]
+    hdfs cacheadmin [-removePool <name>]
+    hdfs cacheadmin [-listPools [-stats] [<name>]]
+    hdfs cacheadmin [-help <command-name>]
 
 See the [HDFS Cache Administration Documentation](./CentralizedCacheManagement.html#cacheadmin_command-line_interface) for more information.
 
@@ -298,8 +337,8 @@ See the [HDFS Cache Administration Documentation](./CentralizedCacheManagement.h
 Usage:
 
       hdfs crypto -createZone -keyName <keyName> -path <path>
-      hdfs crypto -help <command-name>
       hdfs crypto -listZones
+      hdfs crypto -help <command-name>
 
 See the [HDFS Transparent Encryption Documentation](./TransparentEncryption.html#crypto_command-line_interface) for more information.
 
@@ -319,44 +358,43 @@ Runs a HDFS datanode.
 
 Usage:
 
-        hdfs dfsadmin [GENERIC_OPTIONS]
-              [-report [-live] [-dead] [-decommissioning]]
-              [-safemode enter | leave | get | wait | forceExit]
-              [-saveNamespace]
-              [-rollEdits]
-              [-restoreFailedStorage true |false |check]
-              [-refreshNodes]
-              [-setQuota <quota> <dirname>...<dirname>]
-              [-clrQuota <dirname>...<dirname>]
-              [-setSpaceQuota <quota> [-storageType <storagetype>] <dirname>...<dirname>]
-              [-clrSpaceQuota [-storageType <storagetype>] <dirname>...<dirname>]
-              [-finalizeUpgrade]
-              [-rollingUpgrade [<query> |<prepare> |<finalize>]]
-              [-metasave filename]
-              [-refreshServiceAcl]
-              [-refreshUserToGroupsMappings]
-              [-refreshSuperUserGroupsConfiguration]
-              [-refreshCallQueue]
-              [-refresh <host:ipc_port> <key> [arg1..argn]]
-              [-reconfig <datanode |...> <host:ipc_port> <start |status>]
-              [-printTopology]
-              [-refreshNamenodes datanodehost:port]
-              [-deleteBlockPool datanode-host:port blockpoolId [force]]
-              [-setBalancerBandwidth <bandwidth in bytes per second>]
-              [-getBalancerBandwidth <datanode_host:ipc_port>]
-              [-allowSnapshot <snapshotDir>]
-              [-disallowSnapshot <snapshotDir>]
-              [-fetchImage <local directory>]
-              [-shutdownDatanode <datanode_host:ipc_port> [upgrade]]
-              [-getDatanodeInfo <datanode_host:ipc_port>]
-              [-triggerBlockReport [-incremental] <datanode_host:ipc_port>]
-              [-help [cmd]]
+        hdfs dfsadmin [-report [-live] [-dead] [-decommissioning]]
+        hdfs dfsadmin [-safemode enter | leave | get | wait | forceExit]
+        hdfs dfsadmin [-saveNamespace [-beforeShutdown]]
+        hdfs dfsadmin [-rollEdits]
+        hdfs dfsadmin [-restoreFailedStorage true |false |check]
+        hdfs dfsadmin [-refreshNodes]
+        hdfs dfsadmin [-setQuota <quota> <dirname>...<dirname>]
+        hdfs dfsadmin [-clrQuota <dirname>...<dirname>]
+        hdfs dfsadmin [-setSpaceQuota <quota> [-storageType <storagetype>] <dirname>...<dirname>]
+        hdfs dfsadmin [-clrSpaceQuota [-storageType <storagetype>] <dirname>...<dirname>]
+        hdfs dfsadmin [-finalizeUpgrade]
+        hdfs dfsadmin [-rollingUpgrade [<query> |<prepare> |<finalize>]]
+        hdfs dfsadmin [-refreshServiceAcl]
+        hdfs dfsadmin [-refreshUserToGroupsMappings]
+        hdfs dfsadmin [-refreshSuperUserGroupsConfiguration]
+        hdfs dfsadmin [-refreshCallQueue]
+        hdfs dfsadmin [-refresh <host:ipc_port> <key> [arg1..argn]]
+        hdfs dfsadmin [-reconfig <namenode|datanode> <host:ipc_port> <start |status |properties>]
+        hdfs dfsadmin [-printTopology]
+        hdfs dfsadmin [-refreshNamenodes datanodehost:port]
+        hdfs dfsadmin [-deleteBlockPool datanode-host:port blockpoolId [force]]
+        hdfs dfsadmin [-setBalancerBandwidth <bandwidth in bytes per second>]
+        hdfs dfsadmin [-getBalancerBandwidth <datanode_host:ipc_port>]
+        hdfs dfsadmin [-fetchImage <local directory>]
+        hdfs dfsadmin [-allowSnapshot <snapshotDir>]
+        hdfs dfsadmin [-disallowSnapshot <snapshotDir>]
+        hdfs dfsadmin [-shutdownDatanode <datanode_host:ipc_port> [upgrade]]
+        hdfs dfsadmin [-getDatanodeInfo <datanode_host:ipc_port>]
+        hdfs dfsadmin [-metasave filename]
+        hdfs dfsadmin [-triggerBlockReport [-incremental] <datanode_host:ipc_port>]
+        hdfs dfsadmin [-help [cmd]]
 
 | COMMAND\_OPTION | Description |
 |:---- |:---- |
 | `-report` `[-live]` `[-dead]` `[-decommissioning]` | Reports basic filesystem information and statistics, The dfs usage can be different from "du" usage, because it measures raw space used by replication, checksums, snapshots and etc. on all the DNs. Optional flags may be used to filter the list of displayed DataNodes. |
 | `-safemode` enter\|leave\|get\|wait\|forceExit | Safe mode maintenance command. Safe mode is a Namenode state in which it <br/>1. does not accept changes to the name space (read-only) <br/>2. does not replicate or delete blocks. <br/>Safe mode is entered automatically at Namenode startup, and leaves safe mode automatically when the configured minimum percentage of blocks satisfies the minimum replication condition. If Namenode detects any anomaly then it will linger in safe mode till that issue is resolved. If that anomaly is the consequence of a deliberate action, then administrator can use -safemode forceExit to exit safe mode. The cases where forceExit may be required are<br/> 1. Namenode metadata is not consistent. If Namenode detects that metadata has been modified out of band and can cause data loss, then Namenode will enter forceExit state. At that point user can either restart Namenode with correct metadata files or forceExit (if data loss is acceptable).<br/>2. Rollback c
 auses metadata to be replaced and rarely it can trigger safe mode forceExit state in Namenode. In that case you may proceed by issuing -safemode forceExit.<br/> Safe mode can also be entered manually, but then it can only be turned off manually as well. |
-| `-saveNamespace` | Save current namespace into storage directories and reset edits log. Requires safe mode. |
+| `-saveNamespace` `\[-beforeShutdown\]` | Save current namespace into storage directories and reset edits log. Requires safe mode. If the "beforeShutdown" option is given, the NameNode does a checkpoint if and only if no checkpoint has been done during a time window (a configurable number of checkpoint periods). This is usually used before shutting down the NameNode to prevent potential fsimage/editlog corruption. |
 | `-rollEdits` | Rolls the edit log on the active NameNode. |
 | `-restoreFailedStorage` true\|false\|check | This option will turn on/off automatic attempt to restore failed storage replicas. If a failed storage becomes available again the system will attempt to restore edits and/or fsimage during checkpoint. 'check' option will return current setting. |
 | `-refreshNodes` | Re-read the hosts and exclude files to update the set of Datanodes that are allowed to connect to the Namenode and those that should be decommissioned or recommissioned. |
@@ -366,23 +404,23 @@ Usage:
 | `-clrSpaceQuota` `[-storageType <storagetype>]` \<dirname\>...\<dirname\> | See [HDFS Quotas Guide](../hadoop-hdfs/HdfsQuotaAdminGuide.html#Administrative_Commands) for the detail. |
 | `-finalizeUpgrade` | Finalize upgrade of HDFS. Datanodes delete their previous version working directories, followed by Namenode doing the same. This completes the upgrade process. |
 | `-rollingUpgrade` [\<query\>\|\<prepare\>\|\<finalize\>] | See [Rolling Upgrade document](../hadoop-hdfs/HdfsRollingUpgrade.html#dfsadmin_-rollingUpgrade) for the detail. |
-| `-metasave` filename | Save Namenode's primary data structures to *filename* in the directory specified by hadoop.log.dir property. *filename* is overwritten if it exists. *filename* will contain one line for each of the following<br/>1. Datanodes heart beating with Namenode<br/>2. Blocks waiting to be replicated<br/>3. Blocks currently being replicated<br/>4. Blocks waiting to be deleted |
 | `-refreshServiceAcl` | Reload the service-level authorization policy file. |
 | `-refreshUserToGroupsMappings` | Refresh user-to-groups mappings. |
 | `-refreshSuperUserGroupsConfiguration` | Refresh superuser proxy groups mappings |
 | `-refreshCallQueue` | Reload the call queue from config. |
 | `-refresh` \<host:ipc\_port\> \<key\> [arg1..argn] | Triggers a runtime-refresh of the resource specified by \<key\> on \<host:ipc\_port\>. All other args after are sent to the host. |
-| `-reconfig` \<datanode \|...\> \<host:ipc\_port\> \<start\|status\> | Start reconfiguration or get the status of an ongoing reconfiguration. The second parameter specifies the node type. Currently, only reloading DataNode's configuration is supported. |
+| `-reconfig` \<datanode \|namenode\> \<host:ipc\_port\> \<start\|status\|properties\> | Starts reconfiguration or gets the status of an ongoing reconfiguration, or gets a list of reconfigurable properties. The second parameter specifies the node type. |
 | `-printTopology` | Print a tree of the racks and their nodes as reported by the Namenode |
 | `-refreshNamenodes` datanodehost:port | For the given datanode, reloads the configuration files, stops serving the removed block-pools and starts serving new block-pools. |
 | `-deleteBlockPool` datanode-host:port blockpoolId [force] | If force is passed, block pool directory for the given blockpool id on the given datanode is deleted along with its contents, otherwise the directory is deleted only if it is empty. The command will fail if datanode is still serving the block pool. Refer to refreshNamenodes to shutdown a block pool service on a datanode. |
 | `-setBalancerBandwidth` \<bandwidth in bytes per second\> | Changes the network bandwidth used by each datanode during HDFS block balancing. \<bandwidth\> is the maximum number of bytes per second that will be used by each datanode. This value overrides the dfs.balance.bandwidthPerSec parameter. NOTE: The new value is not persistent on the DataNode. |
 | `-getBalancerBandwidth` \<datanode\_host:ipc\_port\> | Get the network bandwidth(in bytes per second) for the given datanode. This is the maximum network bandwidth used by the datanode during HDFS block balancing.|
+| `-fetchImage` \<local directory\> | Downloads the most recent fsimage from the NameNode and saves it in the specified local directory. |
 | `-allowSnapshot` \<snapshotDir\> | Allowing snapshots of a directory to be created. If the operation completes successfully, the directory becomes snapshottable. See the [HDFS Snapshot Documentation](./HdfsSnapshots.html) for more information. |
 | `-disallowSnapshot` \<snapshotDir\> | Disallowing snapshots of a directory to be created. All snapshots of the directory must be deleted before disallowing snapshots. See the [HDFS Snapshot Documentation](./HdfsSnapshots.html) for more information. |
-| `-fetchImage` \<local directory\> | Downloads the most recent fsimage from the NameNode and saves it in the specified local directory. |
 | `-shutdownDatanode` \<datanode\_host:ipc\_port\> [upgrade] | Submit a shutdown request for the given datanode. See [Rolling Upgrade document](./HdfsRollingUpgrade.html#dfsadmin_-shutdownDatanode) for the detail. |
 | `-getDatanodeInfo` \<datanode\_host:ipc\_port\> | Get the information about the given datanode. See [Rolling Upgrade document](./HdfsRollingUpgrade.html#dfsadmin_-getDatanodeInfo) for the detail. |
+| `-metasave` filename | Save Namenode's primary data structures to *filename* in the directory specified by hadoop.log.dir property. *filename* is overwritten if it exists. *filename* will contain one line for each of the following<br/>1. Datanodes heart beating with Namenode<br/>2. Blocks waiting to be replicated<br/>3. Blocks currently being replicated<br/>4. Blocks waiting to be deleted |
 | `-triggerBlockReport` `[-incremental]` \<datanode\_host:ipc\_port\> | Trigger a block report for the given datanode. If 'incremental' is specified, it will be otherwise, it will be a full block report. |
 | `-help` [cmd] | Displays help for the given command or all commands if none is specified. |
 
@@ -411,12 +449,13 @@ Runs the ErasureCoding CLI. See [HDFS ErasureCoding](./HDFSErasureCoding.html#Ad
 
 Usage:
 
-        hdfs haadmin -checkHealth <serviceId>
+        hdfs haadmin -transitionToActive <serviceId> [--forceactive]
+        hdfs haadmin -transitionToStandby <serviceId>
         hdfs haadmin -failover [--forcefence] [--forceactive] <serviceId> <serviceId>
         hdfs haadmin -getServiceState <serviceId>
+        hdfs haadmin -checkHealth <serviceId>
         hdfs haadmin -help <command>
-        hdfs haadmin -transitionToActive <serviceId> [--forceactive]
-        hdfs haadmin -transitionToStandby <serviceId>
+
 
 | COMMAND\_OPTION | Description |
 |:---- |:---- |
@@ -425,6 +464,7 @@ Usage:
 | `-getServiceState` | determine whether the given NameNode is Active or Standby |
 | `-transitionToActive` | transition the state of the given NameNode to Active (Warning: No fencing is done) |
 | `-transitionToStandby` | transition the state of the given NameNode to Standby (Warning: No fencing is done) |
+| `-help` [cmd] | Displays help for the given command or all commands if none is specified. |
 
 See [HDFS HA with NFS](./HDFSHighAvailabilityWithNFS.html#Administrative_commands) or [HDFS HA with QJM](./HDFSHighAvailabilityWithQJM.html#Administrative_commands) for more information on this command.
 
@@ -509,9 +549,16 @@ Runs the HDFS secondary namenode. See [Secondary Namenode](./HdfsUserGuide.html#
 
 ### `storagepolicies`
 
-Usage: `hdfs storagepolicies`
+Usage:
+
+      hdfs storagepolicies
+          [-listPolicies]
+          [-setStoragePolicy -path <path> -policy <policy>]
+          [-getStoragePolicy -path <path>]
+          [-unsetStoragePolicy -path <path>]
+          [-help <command-name>]
 
-Lists out all storage policies. See the [HDFS Storage Policy Documentation](./ArchivalStorage.html) for more information.
+Lists out all/Gets/sets/unsets storage policies. See the [HDFS Storage Policy Documentation](./ArchivalStorage.html) for more information.
 
 ### `zkfc`
 


[26/46] hadoop git commit: YARN-4593 Deadlock in AbstractService.getConfig() (stevel)

Posted by ae...@apache.org.
YARN-4593 Deadlock in AbstractService.getConfig() (stevel)


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

Branch: refs/heads/HDFS-7240
Commit: 605fdcbb81687c73ba91a3bd0d607cabd3dc5a67
Parents: 5a725f0
Author: Steve Loughran <st...@apache.org>
Authored: Wed Mar 16 14:35:13 2016 +0000
Committer: Steve Loughran <st...@apache.org>
Committed: Wed Mar 16 14:35:24 2016 +0000

----------------------------------------------------------------------
 .../src/main/java/org/apache/hadoop/service/AbstractService.java   | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/605fdcbb/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/service/AbstractService.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/service/AbstractService.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/service/AbstractService.java
index a7691f6..1327683 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/service/AbstractService.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/service/AbstractService.java
@@ -399,7 +399,7 @@ public abstract class AbstractService implements Service {
   }
 
   @Override
-  public synchronized Configuration getConfig() {
+  public Configuration getConfig() {
     return config;
   }
 


[09/46] hadoop git commit: YARN-4545. Allow YARN distributed shell to use ATS v1.5 APIs. Li Lu via junping_du

Posted by ae...@apache.org.
YARN-4545. Allow YARN distributed shell to use ATS v1.5 APIs. Li Lu via junping_du


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

Branch: refs/heads/HDFS-7240
Commit: f291d82cd49c04a81380bc45c97c279d791b571c
Parents: 658ee95
Author: Junping Du <ju...@apache.org>
Authored: Mon Mar 14 08:28:38 2016 -0700
Committer: Junping Du <ju...@apache.org>
Committed: Mon Mar 14 08:28:38 2016 -0700

----------------------------------------------------------------------
 hadoop-project/pom.xml                          |  13 ++
 .../pom.xml                                     |  21 ++++
 .../distributedshell/ApplicationMaster.java     |  68 +++++++----
 .../DistributedShellTimelinePlugin.java         |  79 ++++++++++++
 .../distributedshell/package-info.java          |  19 +++
 .../distributedshell/TestDistributedShell.java  | 120 ++++++++++++++++---
 .../yarn/util/timeline/TimelineUtils.java       |  35 ++++++
 .../hadoop/yarn/server/MiniYARNCluster.java     |  10 +-
 .../yarn/server/timeline/TimelineVersion.java   |  31 +++++
 .../server/timeline/TimelineVersionWatcher.java |  47 ++++++++
 .../pom.xml                                     |  16 +++
 .../server/timeline/PluginStoreTestUtils.java   |  51 +++++++-
 12 files changed, 469 insertions(+), 41 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/f291d82c/hadoop-project/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-project/pom.xml b/hadoop-project/pom.xml
index f23b46e..3362c11 100644
--- a/hadoop-project/pom.xml
+++ b/hadoop-project/pom.xml
@@ -298,6 +298,19 @@
 
       <dependency>
         <groupId>org.apache.hadoop</groupId>
+        <artifactId>hadoop-yarn-server-timeline-pluginstorage</artifactId>
+        <version>${project.version}</version>
+      </dependency>
+
+      <dependency>
+        <groupId>org.apache.hadoop</groupId>
+        <artifactId>hadoop-yarn-server-timeline-pluginstorage</artifactId>
+        <type>test-jar</type>
+        <version>${project.version}</version>
+      </dependency>
+
+      <dependency>
+        <groupId>org.apache.hadoop</groupId>
          <artifactId>hadoop-mapreduce-client-jobclient</artifactId>
         <version>${project.version}</version>
         <type>test-jar</type>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f291d82c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/pom.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/pom.xml
index 09a56ea..c118603 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/pom.xml
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/pom.xml
@@ -121,6 +121,27 @@
       <artifactId>mockito-all</artifactId>
       <scope>test</scope>
     </dependency>
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-yarn-server-timeline-pluginstorage</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-yarn-server-timeline-pluginstorage</artifactId>
+      <type>test-jar</type>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-hdfs</artifactId>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-hdfs</artifactId>
+      <scope>test</scope>
+      <type>test-jar</type>
+    </dependency>
   </dependencies>
 
   <build>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f291d82c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/main/java/org/apache/hadoop/yarn/applications/distributedshell/ApplicationMaster.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/main/java/org/apache/hadoop/yarn/applications/distributedshell/ApplicationMaster.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/main/java/org/apache/hadoop/yarn/applications/distributedshell/ApplicationMaster.java
index 95dbddc..0f82903 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/main/java/org/apache/hadoop/yarn/applications/distributedshell/ApplicationMaster.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/main/java/org/apache/hadoop/yarn/applications/distributedshell/ApplicationMaster.java
@@ -88,6 +88,7 @@ import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.ResourceRequest;
 import org.apache.hadoop.yarn.api.records.URL;
 import org.apache.hadoop.yarn.api.records.timeline.TimelineEntity;
+import org.apache.hadoop.yarn.api.records.timeline.TimelineEntityGroupId;
 import org.apache.hadoop.yarn.api.records.timeline.TimelineEvent;
 import org.apache.hadoop.yarn.api.records.timeline.TimelinePutResponse;
 import org.apache.hadoop.yarn.client.api.AMRMClient.ContainerRequest;
@@ -99,6 +100,7 @@ import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.security.AMRMTokenIdentifier;
 import org.apache.hadoop.yarn.util.ConverterUtils;
+import org.apache.hadoop.yarn.util.timeline.TimelineUtils;
 import org.apache.log4j.LogManager;
 
 import com.google.common.annotations.VisibleForTesting;
@@ -277,6 +279,9 @@ public class ApplicationMaster {
   // Timeline Client
   @VisibleForTesting
   TimelineClient timelineClient;
+  static final String CONTAINER_ENTITY_GROUP_ID = "CONTAINERS";
+  static final String APPID_TIMELINE_FILTER_NAME = "appId";
+  static final String USER_TIMELINE_FILTER_NAME = "user";
 
   private final String linux_bash_command = "bash";
   private final String windows_command = "cmd /c";
@@ -904,7 +909,7 @@ public class ApplicationMaster {
         applicationMaster.nmClientAsync.getContainerStatusAsync(containerId, container.getNodeId());
       }
       if(applicationMaster.timelineClient != null) {
-        ApplicationMaster.publishContainerStartEvent(
+        applicationMaster.publishContainerStartEvent(
             applicationMaster.timelineClient, container,
             applicationMaster.domainId, applicationMaster.appSubmitterUgi);
       }
@@ -1120,15 +1125,17 @@ public class ApplicationMaster {
       org.apache.commons.io.IOUtils.closeQuietly(ds);
     }
   }
-  
-  private static void publishContainerStartEvent(
-      final TimelineClient timelineClient, Container container, String domainId,
-      UserGroupInformation ugi) {
+
+  private void publishContainerStartEvent(
+      final TimelineClient timelineClient, final Container container,
+      String domainId, UserGroupInformation ugi) {
     final TimelineEntity entity = new TimelineEntity();
     entity.setEntityId(container.getId().toString());
     entity.setEntityType(DSEntity.DS_CONTAINER.toString());
     entity.setDomainId(domainId);
-    entity.addPrimaryFilter("user", ugi.getShortUserName());
+    entity.addPrimaryFilter(USER_TIMELINE_FILTER_NAME, ugi.getShortUserName());
+    entity.addPrimaryFilter(APPID_TIMELINE_FILTER_NAME, container.getId()
+        .getApplicationAttemptId().getApplicationId().toString());
     TimelineEvent event = new TimelineEvent();
     event.setTimestamp(System.currentTimeMillis());
     event.setEventType(DSEvent.DS_CONTAINER_START.toString());
@@ -1137,28 +1144,27 @@ public class ApplicationMaster {
     entity.addEvent(event);
 
     try {
-      ugi.doAs(new PrivilegedExceptionAction<TimelinePutResponse>() {
-        @Override
-        public TimelinePutResponse run() throws Exception {
-          return processTimelineResponseErrors(
-              timelineClient.putEntities(entity));
-        }
-      });
-    } catch (Exception e) {
+      processTimelineResponseErrors(
+          putContainerEntity(timelineClient,
+              container.getId().getApplicationAttemptId(),
+              entity));
+    } catch (YarnException | IOException e) {
       LOG.error("Container start event could not be published for "
-          + container.getId().toString(),
-          e instanceof UndeclaredThrowableException ? e.getCause() : e);
+          + container.getId().toString(), e);
     }
   }
 
-  private static void publishContainerEndEvent(
+  private void publishContainerEndEvent(
       final TimelineClient timelineClient, ContainerStatus container,
       String domainId, UserGroupInformation ugi) {
     final TimelineEntity entity = new TimelineEntity();
     entity.setEntityId(container.getContainerId().toString());
     entity.setEntityType(DSEntity.DS_CONTAINER.toString());
     entity.setDomainId(domainId);
-    entity.addPrimaryFilter("user", ugi.getShortUserName());
+    entity.addPrimaryFilter(USER_TIMELINE_FILTER_NAME, ugi.getShortUserName());
+    entity.addPrimaryFilter(APPID_TIMELINE_FILTER_NAME,
+        container.getContainerId().getApplicationAttemptId()
+            .getApplicationId().toString());
     TimelineEvent event = new TimelineEvent();
     event.setTimestamp(System.currentTimeMillis());
     event.setEventType(DSEvent.DS_CONTAINER_END.toString());
@@ -1166,22 +1172,38 @@ public class ApplicationMaster {
     event.addEventInfo("Exit Status", container.getExitStatus());
     entity.addEvent(event);
     try {
-      TimelinePutResponse response = timelineClient.putEntities(entity);
-      processTimelineResponseErrors(response);
+      processTimelineResponseErrors(
+          putContainerEntity(timelineClient,
+              container.getContainerId().getApplicationAttemptId(),
+              entity));
     } catch (YarnException | IOException e) {
       LOG.error("Container end event could not be published for "
           + container.getContainerId().toString(), e);
     }
   }
 
-  private static void publishApplicationAttemptEvent(
+  private TimelinePutResponse putContainerEntity(
+      TimelineClient timelineClient, ApplicationAttemptId currAttemptId,
+      TimelineEntity entity)
+      throws YarnException, IOException {
+    if (TimelineUtils.timelineServiceV1_5Enabled(conf)) {
+      TimelineEntityGroupId groupId = TimelineEntityGroupId.newInstance(
+          currAttemptId.getApplicationId(),
+          CONTAINER_ENTITY_GROUP_ID);
+      return timelineClient.putEntities(currAttemptId, groupId, entity);
+    } else {
+      return timelineClient.putEntities(entity);
+    }
+  }
+
+  private void publishApplicationAttemptEvent(
       final TimelineClient timelineClient, String appAttemptId,
       DSEvent appEvent, String domainId, UserGroupInformation ugi) {
     final TimelineEntity entity = new TimelineEntity();
     entity.setEntityId(appAttemptId);
     entity.setEntityType(DSEntity.DS_APP_ATTEMPT.toString());
     entity.setDomainId(domainId);
-    entity.addPrimaryFilter("user", ugi.getShortUserName());
+    entity.addPrimaryFilter(USER_TIMELINE_FILTER_NAME, ugi.getShortUserName());
     TimelineEvent event = new TimelineEvent();
     event.setEventType(appEvent.toString());
     event.setTimestamp(System.currentTimeMillis());
@@ -1197,7 +1219,7 @@ public class ApplicationMaster {
     }
   }
 
-  private static TimelinePutResponse processTimelineResponseErrors(
+  private TimelinePutResponse processTimelineResponseErrors(
       TimelinePutResponse response) {
     List<TimelinePutResponse.TimelinePutError> errors = response.getErrors();
     if (errors.size() == 0) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f291d82c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/main/java/org/apache/hadoop/yarn/applications/distributedshell/DistributedShellTimelinePlugin.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/main/java/org/apache/hadoop/yarn/applications/distributedshell/DistributedShellTimelinePlugin.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/main/java/org/apache/hadoop/yarn/applications/distributedshell/DistributedShellTimelinePlugin.java
new file mode 100644
index 0000000..55fbd60
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/main/java/org/apache/hadoop/yarn/applications/distributedshell/DistributedShellTimelinePlugin.java
@@ -0,0 +1,79 @@
+/**
+ * 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.applications.distributedshell;
+
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.timeline.TimelineEntityGroupId;
+import org.apache.hadoop.yarn.server.timeline.NameValuePair;
+import org.apache.hadoop.yarn.server.timeline.TimelineEntityGroupPlugin;
+import org.apache.hadoop.yarn.util.ConverterUtils;
+
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.Set;
+import java.util.SortedSet;
+
+/**
+ * Timeline v1.5 reader plugin for YARN distributed shell. It tranlsates an
+ * incoming getEntity request to a set of related timeline entity groups, via
+ * the information provided in the primary filter or entity id field.
+ */
+public class DistributedShellTimelinePlugin extends TimelineEntityGroupPlugin {
+
+  @Override
+  public Set<TimelineEntityGroupId> getTimelineEntityGroupId(String entityType,
+      NameValuePair primaryFilter, Collection<NameValuePair> secondaryFilters) {
+    if (ApplicationMaster.DSEntity.DS_CONTAINER.toString().equals(entityType)) {
+      if (primaryFilter == null) {
+        return null;
+      }
+      return toEntityGroupId(primaryFilter.getValue().toString());
+    }
+    return null;
+  }
+
+  @Override
+  public Set<TimelineEntityGroupId> getTimelineEntityGroupId(String entityId,
+      String entityType) {
+    if (ApplicationMaster.DSEntity.DS_CONTAINER.toString().equals(entityId)) {
+      ContainerId containerId = ConverterUtils.toContainerId(entityId);
+      ApplicationId appId = containerId.getApplicationAttemptId()
+          .getApplicationId();
+      return toEntityGroupId(appId.toString());
+    }
+    return null;
+  }
+
+  @Override
+  public Set<TimelineEntityGroupId> getTimelineEntityGroupId(String entityType,
+      SortedSet<String> entityIds, Set<String> eventTypes) {
+    // Right now this method is not used by TimelineEntityGroupPlugin
+    return null;
+  }
+
+  private Set<TimelineEntityGroupId> toEntityGroupId(String strAppId) {
+    ApplicationId appId = ConverterUtils.toApplicationId(strAppId);
+    TimelineEntityGroupId groupId = TimelineEntityGroupId.newInstance(
+        appId, ApplicationMaster.CONTAINER_ENTITY_GROUP_ID);
+    Set<TimelineEntityGroupId> result = new HashSet<>();
+    result.add(groupId);
+    return result;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f291d82c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/main/java/org/apache/hadoop/yarn/applications/distributedshell/package-info.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/main/java/org/apache/hadoop/yarn/applications/distributedshell/package-info.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/main/java/org/apache/hadoop/yarn/applications/distributedshell/package-info.java
new file mode 100644
index 0000000..299a286
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/main/java/org/apache/hadoop/yarn/applications/distributedshell/package-info.java
@@ -0,0 +1,19 @@
+/**
+ * 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.applications.distributedshell;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f291d82c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/test/java/org/apache/hadoop/yarn/applications/distributedshell/TestDistributedShell.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/test/java/org/apache/hadoop/yarn/applications/distributedshell/TestDistributedShell.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/test/java/org/apache/hadoop/yarn/applications/distributedshell/TestDistributedShell.java
index 3197875..6536050 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/test/java/org/apache/hadoop/yarn/applications/distributedshell/TestDistributedShell.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/test/java/org/apache/hadoop/yarn/applications/distributedshell/TestDistributedShell.java
@@ -36,12 +36,19 @@ import java.util.concurrent.atomic.AtomicBoolean;
 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.FileContext;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.RemoteIterator;
+import org.apache.hadoop.hdfs.HdfsConfiguration;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.net.ServerSocketUtil;
 import org.apache.hadoop.util.JarFinder;
 import org.apache.hadoop.util.Shell;
+import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationReport;
 import org.apache.hadoop.yarn.api.records.YarnApplicationState;
 import org.apache.hadoop.yarn.api.records.timeline.TimelineDomain;
@@ -50,29 +57,50 @@ import org.apache.hadoop.yarn.client.api.YarnClient;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.server.MiniYARNCluster;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler;
+import org.apache.hadoop.yarn.server.timeline.PluginStoreTestUtils;
+import org.apache.hadoop.yarn.server.timeline.NameValuePair;
+import org.apache.hadoop.yarn.server.timeline.TimelineVersion;
+import org.apache.hadoop.yarn.server.timeline.TimelineVersionWatcher;
+import org.apache.hadoop.yarn.util.ConverterUtils;
 import org.junit.After;
 import org.junit.Assert;
 import org.junit.Before;
+import org.junit.Rule;
 import org.junit.Test;
+import org.junit.rules.Timeout;
 
 public class TestDistributedShell {
 
   private static final Log LOG =
       LogFactory.getLog(TestDistributedShell.class);
 
-  protected MiniYARNCluster yarnCluster = null;  
+  protected MiniYARNCluster yarnCluster = null;
+  protected MiniDFSCluster hdfsCluster = null;
+  private FileSystem fs = null;
   protected YarnConfiguration conf = null;
   private static final int NUM_NMS = 1;
+  private static final float DEFAULT_TIMELINE_VERSION = 1.0f;
 
   protected final static String APPMASTER_JAR =
       JarFinder.getJar(ApplicationMaster.class);
 
+  @Rule
+  public TimelineVersionWatcher timelineVersionWatcher
+      = new TimelineVersionWatcher();
+  @Rule
+  public Timeout globalTimeout = new Timeout(90000);
+
   @Before
   public void setup() throws Exception {
-    setupInternal(NUM_NMS);
+    setupInternal(NUM_NMS, timelineVersionWatcher.getTimelineVersion());
   }
 
   protected void setupInternal(int numNodeManager) throws Exception {
+    setupInternal(numNodeManager, DEFAULT_TIMELINE_VERSION);
+  }
+
+  private void setupInternal(int numNodeManager, float timelineVersion)
+      throws Exception {
 
     LOG.info("Starting up YARN cluster");
     
@@ -84,6 +112,26 @@ public class TestDistributedShell {
     conf.setBoolean(YarnConfiguration.NODE_LABELS_ENABLED, true);
     conf.set("mapreduce.jobhistory.address",
         "0.0.0.0:" + ServerSocketUtil.getPort(10021, 10));
+
+    // ATS version specific settings
+    if (timelineVersion == 1.0f) {
+      conf.setFloat(YarnConfiguration.TIMELINE_SERVICE_VERSION, 1.0f);
+      conf.set(CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY,
+          CommonConfigurationKeysPublic.FS_DEFAULT_NAME_DEFAULT);
+    } else if (timelineVersion == 1.5f) {
+      if (hdfsCluster == null) {
+        HdfsConfiguration hdfsConfig = new HdfsConfiguration();
+        hdfsCluster = new MiniDFSCluster.Builder(hdfsConfig)
+            .numDataNodes(1).build();
+      }
+      fs = hdfsCluster.getFileSystem();
+      PluginStoreTestUtils.prepareFileSystemForPluginStore(fs);
+      PluginStoreTestUtils.prepareConfiguration(conf, hdfsCluster);
+      conf.set(YarnConfiguration.TIMELINE_SERVICE_ENTITY_GROUP_PLUGIN_CLASSES,
+          DistributedShellTimelinePlugin.class.getName());
+    } else {
+      Assert.fail("Wrong timeline version number: " + timelineVersion);
+    }
     
     if (yarnCluster == null) {
       yarnCluster =
@@ -138,6 +186,13 @@ public class TestDistributedShell {
         yarnCluster = null;
       }
     }
+    if (hdfsCluster != null) {
+      try {
+        hdfsCluster.shutdown();
+      } finally {
+        hdfsCluster = null;
+      }
+    }
     FileContext fsContext = FileContext.getLocalFSFileContext();
     fsContext
         .delete(
@@ -146,16 +201,28 @@ public class TestDistributedShell {
             true);
   }
   
-  @Test(timeout=90000)
+  @Test
   public void testDSShellWithDomain() throws Exception {
     testDSShell(true);
   }
 
-  @Test(timeout=90000)
+  @Test
   public void testDSShellWithoutDomain() throws Exception {
     testDSShell(false);
   }
 
+  @Test
+  @TimelineVersion(1.5f)
+  public void testDSShellWithoutDomainV1_5() throws Exception {
+    testDSShell(false);
+  }
+
+  @Test
+  @TimelineVersion(1.5f)
+  public void testDSShellWithDomainV1_5() throws Exception {
+    testDSShell(true);
+  }
+
   public void testDSShell(boolean haveDomain) throws Exception {
     String[] args = {
         "--jar",
@@ -239,6 +306,24 @@ public class TestDistributedShell {
     LOG.info("Client run completed. Result=" + result);
     Assert.assertTrue(result.get());
 
+    if (timelineVersionWatcher.getTimelineVersion() == 1.5f) {
+      long scanInterval = conf.getLong(
+          YarnConfiguration.TIMELINE_SERVICE_ENTITYGROUP_FS_STORE_SCAN_INTERVAL_SECONDS,
+          YarnConfiguration.TIMELINE_SERVICE_ENTITYGROUP_FS_STORE_SCAN_INTERVAL_SECONDS_DEFAULT
+      );
+      Path doneDir = new Path(
+          YarnConfiguration.TIMELINE_SERVICE_ENTITYGROUP_FS_STORE_DONE_DIR_DEFAULT
+      );
+      // Wait till the data is moved to done dir, or timeout and fail
+      while (true) {
+        RemoteIterator<FileStatus> iterApps = fs.listStatusIterator(doneDir);
+        if (iterApps.hasNext()) {
+          break;
+        }
+        Thread.sleep(scanInterval * 2);
+      }
+    }
+
     TimelineDomain domain = null;
     if (haveDomain) {
       domain = yarnCluster.getApplicationHistoryServer()
@@ -265,11 +350,18 @@ public class TestDistributedShell {
       Assert.assertEquals("DEFAULT",
           entitiesAttempts.getEntities().get(0).getDomainId());
     }
+    String currAttemptEntityId
+        = entitiesAttempts.getEntities().get(0).getEntityId();
+    ApplicationAttemptId attemptId
+        = ConverterUtils.toApplicationAttemptId(currAttemptEntityId);
+    NameValuePair primaryFilter = new NameValuePair(
+        ApplicationMaster.APPID_TIMELINE_FILTER_NAME,
+        attemptId.getApplicationId().toString());
     TimelineEntities entities = yarnCluster
         .getApplicationHistoryServer()
         .getTimelineStore()
         .getEntities(ApplicationMaster.DSEntity.DS_CONTAINER.toString(), null,
-            null, null, null, null, null, null, null, null);
+            null, null, null, null, primaryFilter, null, null, null);
     Assert.assertNotNull(entities);
     Assert.assertEquals(2, entities.getEntities().size());
     Assert.assertEquals(entities.getEntities().get(0).getEntityType()
@@ -341,7 +433,7 @@ public class TestDistributedShell {
 
   }
 
-  @Test(timeout=90000)
+  @Test
   public void testDSRestartWithPreviousRunningContainers() throws Exception {
     String[] args = {
         "--jar",
@@ -376,7 +468,7 @@ public class TestDistributedShell {
    * how many attempt failures for previous 2.5 seconds.
    * The application is expected to be successful.
    */
-  @Test(timeout=90000)
+  @Test
   public void testDSAttemptFailuresValidityIntervalSucess() throws Exception {
     String[] args = {
         "--jar",
@@ -414,7 +506,7 @@ public class TestDistributedShell {
    * how many attempt failure for previous 15 seconds.
    * The application is expected to be fail.
    */
-  @Test(timeout=90000)
+  @Test
   public void testDSAttemptFailuresValidityIntervalFailed() throws Exception {
     String[] args = {
         "--jar",
@@ -446,7 +538,7 @@ public class TestDistributedShell {
       Assert.assertFalse(result);
     }
 
-  @Test(timeout=90000)
+  @Test
   public void testDSShellWithCustomLogPropertyFile() throws Exception {
     final File basedir =
         new File("target", TestDistributedShell.class.getName());
@@ -541,7 +633,7 @@ public class TestDistributedShell {
     verifyContainerLog(2, expectedContent, false, "");
   }
 
-  @Test(timeout=90000)
+  @Test
   public void testDSShellWithMultipleArgs() throws Exception {
     String[] args = {
         "--jar",
@@ -575,7 +667,7 @@ public class TestDistributedShell {
     verifyContainerLog(4, expectedContent, false, "");
   }
 
-  @Test(timeout=90000)
+  @Test
   public void testDSShellWithShellScript() throws Exception {
     final File basedir =
         new File("target", TestDistributedShell.class.getName());
@@ -623,7 +715,7 @@ public class TestDistributedShell {
     verifyContainerLog(1, expectedContent, false, "");
   }
 
-  @Test(timeout=90000)
+  @Test
   public void testDSShellWithInvalidArgs() throws Exception {
     Client client = new Client(new Configuration(yarnCluster.getConfig()));
 
@@ -785,7 +877,7 @@ public class TestDistributedShell {
     }
   }
 
-  @Test(timeout=90000)
+  @Test
   public void testContainerLaunchFailureHandling() throws Exception {
     String[] args = {
       "--jar",
@@ -813,7 +905,7 @@ public class TestDistributedShell {
 
   }
 
-  @Test(timeout=90000)
+  @Test
   public void testDebugFlag() throws Exception {
     String[] args = {
         "--jar",

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f291d82c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/timeline/TimelineUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/timeline/TimelineUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/timeline/TimelineUtils.java
index 4f838e6..a794e97 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/timeline/TimelineUtils.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/timeline/TimelineUtils.java
@@ -86,6 +86,41 @@ public class TimelineUtils {
     }
   }
 
+  /**
+   * Returns whether the timeline service is enabled via configuration.
+   *
+   * @param conf the configuration
+   * @return whether the timeline service is enabled.
+   */
+  public static boolean timelineServiceEnabled(Configuration conf) {
+    return conf.getBoolean(YarnConfiguration.TIMELINE_SERVICE_ENABLED,
+        YarnConfiguration.DEFAULT_TIMELINE_SERVICE_ENABLED);
+  }
+
+  /**
+   * Returns the timeline service version. It does not check whether the
+   * timeline service itself is enabled.
+   *
+   * @param conf the configuration
+   * @return the timeline service version as a float.
+   */
+  public static float getTimelineServiceVersion(Configuration conf) {
+    return conf.getFloat(YarnConfiguration.TIMELINE_SERVICE_VERSION,
+        YarnConfiguration.DEFAULT_TIMELINE_SERVICE_VERSION);
+  }
+
+  /**
+   * Returns whether the timeline service v.1.5 is enabled via configuration.
+   *
+   * @param conf the configuration
+   * @return whether the timeline service v.1.5 is enabled. V.1.5 refers to a
+   * version equal to 1.5.
+   */
+  public static boolean timelineServiceV1_5Enabled(Configuration conf) {
+    return timelineServiceEnabled(conf) &&
+        Math.abs(getTimelineServiceVersion(conf) - 1.5) < 0.00001;
+  }
+
   public static TimelineAbout createTimelineAbout(String about) {
     TimelineAbout tsInfo = new TimelineAbout(about);
     tsInfo.setHadoopBuildVersion(VersionInfo.getBuildVersion());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f291d82c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/server/MiniYARNCluster.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/server/MiniYARNCluster.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/server/MiniYARNCluster.java
index 630b7ef..024adc6 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/server/MiniYARNCluster.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/server/MiniYARNCluster.java
@@ -37,7 +37,6 @@ import org.apache.hadoop.ha.HAServiceProtocol;
 import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
 import org.apache.hadoop.service.AbstractService;
 import org.apache.hadoop.service.CompositeService;
-import org.apache.hadoop.service.ServiceStateException;
 import org.apache.hadoop.util.Shell;
 import org.apache.hadoop.util.Shell.ShellCommandExecutor;
 import org.apache.hadoop.yarn.api.protocolrecords.GetClusterMetricsRequest;
@@ -76,6 +75,7 @@ import org.apache.hadoop.yarn.server.timeline.MemoryTimelineStore;
 import org.apache.hadoop.yarn.server.timeline.TimelineStore;
 import org.apache.hadoop.yarn.server.timeline.recovery.MemoryTimelineStateStore;
 import org.apache.hadoop.yarn.server.timeline.recovery.TimelineStateStore;
+import org.apache.hadoop.yarn.util.timeline.TimelineUtils;
 import org.apache.hadoop.yarn.webapp.util.WebAppUtils;
 
 import com.google.common.annotations.VisibleForTesting;
@@ -749,8 +749,12 @@ public class MiniYARNCluster extends CompositeService {
       appHistoryServer = new ApplicationHistoryServer();
       conf.setClass(YarnConfiguration.APPLICATION_HISTORY_STORE,
           MemoryApplicationHistoryStore.class, ApplicationHistoryStore.class);
-      conf.setClass(YarnConfiguration.TIMELINE_SERVICE_STORE,
-          MemoryTimelineStore.class, TimelineStore.class);
+      // Only set memory timeline store if timeline v1.5 is not enabled.
+      // Otherwise, caller has the freedom to choose storage impl.
+      if (!TimelineUtils.timelineServiceV1_5Enabled(conf)) {
+        conf.setClass(YarnConfiguration.TIMELINE_SERVICE_STORE,
+            MemoryTimelineStore.class, TimelineStore.class);
+      }
       conf.setClass(YarnConfiguration.TIMELINE_SERVICE_STATE_STORE_CLASS,
           MemoryTimelineStateStore.class, TimelineStateStore.class);
       if (!useFixedPorts) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f291d82c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/server/timeline/TimelineVersion.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/server/timeline/TimelineVersion.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/server/timeline/TimelineVersion.java
new file mode 100644
index 0000000..57439de
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/server/timeline/TimelineVersion.java
@@ -0,0 +1,31 @@
+/**
+ * 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.timeline;
+
+import java.lang.annotation.ElementType;
+import java.lang.annotation.Retention;
+import java.lang.annotation.RetentionPolicy;
+import java.lang.annotation.Target;
+
+@Retention(value = RetentionPolicy.RUNTIME)
+@Target(value = {ElementType.METHOD})
+public @interface TimelineVersion {
+  float value() default TimelineVersionWatcher.DEFAULT_TIMELINE_VERSION;
+}
+

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f291d82c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/server/timeline/TimelineVersionWatcher.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/server/timeline/TimelineVersionWatcher.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/server/timeline/TimelineVersionWatcher.java
new file mode 100644
index 0000000..b00f13a
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/server/timeline/TimelineVersionWatcher.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.timeline;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.junit.rules.TestWatcher;
+import org.junit.runner.Description;
+
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+public class TimelineVersionWatcher extends TestWatcher {
+  static final float DEFAULT_TIMELINE_VERSION = 1.0f;
+  private TimelineVersion version;
+
+  @Override
+  protected void starting(Description description) {
+    version = description.getAnnotation(TimelineVersion.class);
+  }
+
+  /**
+   * @return the version number of timeline server for the current test (using
+   * timeline server v1.0 by default)
+   */
+  public float getTimelineVersion() {
+    if(version == null) {
+      return DEFAULT_TIMELINE_VERSION;
+    }
+    return version.value();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f291d82c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timeline-pluginstorage/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timeline-pluginstorage/pom.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timeline-pluginstorage/pom.xml
index 71f76d3..1dd301a 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timeline-pluginstorage/pom.xml
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timeline-pluginstorage/pom.xml
@@ -137,4 +137,20 @@
       <artifactId>jackson-databind</artifactId>
     </dependency>
   </dependencies>
+
+  <build>
+    <plugins>
+      <plugin>
+        <artifactId>maven-jar-plugin</artifactId>
+        <executions>
+          <execution>
+            <goals>
+              <goal>test-jar</goal>
+            </goals>
+            <phase>test-compile</phase>
+          </execution>
+        </executions>
+      </plugin>
+    </plugins>
+  </build>
 </project>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f291d82c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timeline-pluginstorage/src/test/java/org/apache/hadoop/yarn/server/timeline/PluginStoreTestUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timeline-pluginstorage/src/test/java/org/apache/hadoop/yarn/server/timeline/PluginStoreTestUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timeline-pluginstorage/src/test/java/org/apache/hadoop/yarn/server/timeline/PluginStoreTestUtils.java
index f529b59..c2c4101 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timeline-pluginstorage/src/test/java/org/apache/hadoop/yarn/server/timeline/PluginStoreTestUtils.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timeline-pluginstorage/src/test/java/org/apache/hadoop/yarn/server/timeline/PluginStoreTestUtils.java
@@ -18,13 +18,16 @@
 package org.apache.hadoop.yarn.server.timeline;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.yarn.api.records.timeline.TimelineEntities;
 import org.apache.hadoop.yarn.api.records.timeline.TimelineEntity;
 import org.apache.hadoop.yarn.api.records.timeline.TimelineEvent;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.server.timeline.security.TimelineACLsManager;
 import org.codehaus.jackson.JsonFactory;
@@ -48,7 +51,53 @@ import java.util.Set;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNotNull;
 
-class PluginStoreTestUtils {
+/**
+ * Utility methods related to the ATS v1.5 plugin storage tests.
+ */
+public class PluginStoreTestUtils {
+
+  /**
+   * For a given file system, setup directories ready to test the plugin storage.
+   *
+   * @param fs a {@link FileSystem} object that the plugin storage will work with
+   * @return the dfsCluster ready to start plugin storage tests.
+   * @throws IOException
+   */
+  public static FileSystem prepareFileSystemForPluginStore(FileSystem fs)
+      throws IOException {
+    Path activeDir = new Path(
+        YarnConfiguration.TIMELINE_SERVICE_ENTITYGROUP_FS_STORE_ACTIVE_DIR_DEFAULT
+    );
+    Path doneDir = new Path(
+        YarnConfiguration.TIMELINE_SERVICE_ENTITYGROUP_FS_STORE_DONE_DIR_DEFAULT
+    );
+
+    fs.mkdirs(activeDir);
+    fs.mkdirs(doneDir);
+    return fs;
+  }
+
+  /**
+   * Prepare configuration for plugin tests. This method will also add the mini
+   * DFS cluster's info to the configuration.
+   * Note: the test program needs to setup the reader plugin by itself.
+   *
+   * @param conf
+   * @param dfsCluster
+   * @return the modified configuration
+   */
+  public static YarnConfiguration prepareConfiguration(YarnConfiguration conf,
+      MiniDFSCluster dfsCluster) {
+    conf.set(CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY,
+        dfsCluster.getURI().toString());
+    conf.setFloat(YarnConfiguration.TIMELINE_SERVICE_VERSION, 1.5f);
+    conf.setLong(
+        YarnConfiguration.TIMELINE_SERVICE_ENTITYGROUP_FS_STORE_SCAN_INTERVAL_SECONDS,
+        1);
+    conf.set(YarnConfiguration.TIMELINE_SERVICE_STORE,
+        EntityGroupFSTimelineStore.class.getName());
+    return conf;
+  }
 
   static FSDataOutputStream createLogFile(Path logPath, FileSystem fs)
       throws IOException {


[19/46] hadoop git commit: TestResourceLocalizationService.testPublicResourceInitializesLocalDir fails Intermittently due to IllegalArgumentException from cleanup (templedf via rkanter)

Posted by ae...@apache.org.
TestResourceLocalizationService.testPublicResourceInitializesLocalDir fails Intermittently due to IllegalArgumentException from cleanup (templedf via rkanter)


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

Branch: refs/heads/HDFS-7240
Commit: 22ca176dfe125a4f7bf38cc63ab8106c40a7a7ba
Parents: d457401
Author: Robert Kanter <rk...@apache.org>
Authored: Tue Mar 15 10:05:10 2016 -0700
Committer: Robert Kanter <rk...@apache.org>
Committed: Tue Mar 15 10:05:10 2016 -0700

----------------------------------------------------------------------
 .../localizer/TestResourceLocalizationService.java                 | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/22ca176d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/TestResourceLocalizationService.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/localizer/TestResourceLocalizationService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/TestResourceLocalizationService.java
index 596f784..bca0752 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/TestResourceLocalizationService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/TestResourceLocalizationService.java
@@ -194,7 +194,7 @@ public class TestResourceLocalizationService {
     conf = null;
     try {
       FileUtils.deleteDirectory(new File(basedir.toString()));
-    } catch (IOException e) {
+    } catch (IOException | IllegalArgumentException e) {
       // ignore
     }
   }


[11/46] hadoop git commit: YARN-4719. Add a helper library to maintain node state and allows common queries. (kasha)

Posted by ae...@apache.org.
YARN-4719. Add a helper library to maintain node state and allows common queries. (kasha)


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

Branch: refs/heads/HDFS-7240
Commit: 20d389ce61eaacb5ddfb329015f50e96ad894f8d
Parents: 5644137
Author: Karthik Kambatla <ka...@cloudera.com>
Authored: Mon Mar 14 14:19:05 2016 -0700
Committer: Karthik Kambatla <ka...@cloudera.com>
Committed: Mon Mar 14 14:19:05 2016 -0700

----------------------------------------------------------------------
 .../scheduler/AbstractYarnScheduler.java        | 170 ++---------
 .../scheduler/ClusterNodeTracker.java           | 300 +++++++++++++++++++
 .../resourcemanager/scheduler/NodeFilter.java   |  33 ++
 .../scheduler/capacity/CapacityScheduler.java   |  86 +++---
 .../scheduler/fair/FSAppAttempt.java            |  13 +-
 .../scheduler/fair/FairScheduler.java           | 104 +++----
 .../scheduler/fifo/FifoScheduler.java           |  32 +-
 .../scheduler/TestAbstractYarnScheduler.java    |  14 +-
 .../scheduler/capacity/TestReservations.java    |  19 +-
 .../scheduler/fair/TestFairScheduler.java       |   7 +-
 .../scheduler/fifo/TestFifoScheduler.java       |  19 +-
 11 files changed, 478 insertions(+), 319 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/20d389ce/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 7ca8671..7d12301 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
@@ -27,11 +27,7 @@ import java.util.Map;
 import java.util.Set;
 import java.util.Timer;
 import java.util.TimerTask;
-import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentMap;
-import java.util.concurrent.locks.ReentrantReadWriteLock;
-import java.util.concurrent.locks.ReentrantReadWriteLock.ReadLock;
-import java.util.concurrent.locks.ReentrantReadWriteLock.WriteLock;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -92,22 +88,10 @@ public abstract class AbstractYarnScheduler
 
   private static final Log LOG = LogFactory.getLog(AbstractYarnScheduler.class);
 
-  // Nodes in the cluster, indexed by NodeId
-  protected Map<NodeId, N> nodes = new ConcurrentHashMap<NodeId, N>();
-
-  // Whole capacity of the cluster
-  protected Resource clusterResource = Resource.newInstance(0, 0);
+  protected final ClusterNodeTracker<N> nodeTracker =
+      new ClusterNodeTracker<>();
 
   protected Resource minimumAllocation;
-  protected Resource maximumAllocation;
-  private Resource configuredMaximumAllocation;
-  private int maxNodeMemory = -1;
-  private int maxNodeVCores = -1;
-  private final ReadLock maxAllocReadLock;
-  private final WriteLock maxAllocWriteLock;
-
-  private boolean useConfiguredMaximumAllocationOnly = true;
-  private long configuredMaximumAllocationWaitTime;
 
   protected RMContext rmContext;
   
@@ -132,9 +116,6 @@ public abstract class AbstractYarnScheduler
    */
   public AbstractYarnScheduler(String name) {
     super(name);
-    ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
-    this.maxAllocReadLock = lock.readLock();
-    this.maxAllocWriteLock = lock.writeLock();
   }
 
   @Override
@@ -142,14 +123,21 @@ public abstract class AbstractYarnScheduler
     nmExpireInterval =
         conf.getInt(YarnConfiguration.RM_NM_EXPIRY_INTERVAL_MS,
           YarnConfiguration.DEFAULT_RM_NM_EXPIRY_INTERVAL_MS);
-    configuredMaximumAllocationWaitTime =
+    long configuredMaximumAllocationWaitTime =
         conf.getLong(YarnConfiguration.RM_WORK_PRESERVING_RECOVERY_SCHEDULING_WAIT_MS,
           YarnConfiguration.DEFAULT_RM_WORK_PRESERVING_RECOVERY_SCHEDULING_WAIT_MS);
+    nodeTracker.setConfiguredMaxAllocationWaitTime(
+        configuredMaximumAllocationWaitTime);
     maxClusterLevelAppPriority = getMaxPriorityFromConf(conf);
     createReleaseCache();
     super.serviceInit(conf);
   }
 
+  @VisibleForTesting
+  public ClusterNodeTracker getNodeTracker() {
+    return nodeTracker;
+  }
+
   public List<Container> getTransferredContainers(
       ApplicationAttemptId currentAttempt) {
     ApplicationId appId = currentAttempt.getApplicationId();
@@ -184,20 +172,21 @@ public abstract class AbstractYarnScheduler
    * Add blacklisted NodeIds to the list that is passed.
    *
    * @param app application attempt.
-   * @param blacklistNodeIdList the list to store blacklisted NodeIds.
    */
-  public void addBlacklistedNodeIdsToList(SchedulerApplicationAttempt app,
-      List<NodeId> blacklistNodeIdList) {
-    for (Map.Entry<NodeId, N> nodeEntry : nodes.entrySet()) {
-      if (SchedulerAppUtils.isBlacklisted(app, nodeEntry.getValue(), LOG)) {
-        blacklistNodeIdList.add(nodeEntry.getKey());
+  public List<N> getBlacklistedNodes(final SchedulerApplicationAttempt app) {
+
+    NodeFilter nodeFilter = new NodeFilter() {
+      @Override
+      public boolean accept(SchedulerNode node) {
+        return SchedulerAppUtils.isBlacklisted(app, node, LOG);
       }
-    }
+    };
+    return nodeTracker.getNodes(nodeFilter);
   }
 
   @Override
   public Resource getClusterResource() {
-    return clusterResource;
+    return nodeTracker.getClusterCapacity();
   }
 
   @Override
@@ -207,22 +196,7 @@ public abstract class AbstractYarnScheduler
 
   @Override
   public Resource getMaximumResourceCapability() {
-    Resource maxResource;
-    maxAllocReadLock.lock();
-    try {
-      if (useConfiguredMaximumAllocationOnly) {
-        if (System.currentTimeMillis() - ResourceManager.getClusterTimeStamp()
-            > configuredMaximumAllocationWaitTime) {
-          useConfiguredMaximumAllocationOnly = false;
-        }
-        maxResource = Resources.clone(configuredMaximumAllocation);
-      } else {
-        maxResource = Resources.clone(maximumAllocation);
-      }
-    } finally {
-      maxAllocReadLock.unlock();
-    }
-    return maxResource;
+    return nodeTracker.getMaxAllowedAllocation();
   }
 
   @Override
@@ -231,15 +205,7 @@ public abstract class AbstractYarnScheduler
   }
 
   protected void initMaximumResourceCapability(Resource maximumAllocation) {
-    maxAllocWriteLock.lock();
-    try {
-      if (this.configuredMaximumAllocation == null) {
-        this.configuredMaximumAllocation = Resources.clone(maximumAllocation);
-        this.maximumAllocation = Resources.clone(maximumAllocation);
-      }
-    } finally {
-      maxAllocWriteLock.unlock();
-    }
+    nodeTracker.setConfiguredMaxAllocation(maximumAllocation);
   }
 
   protected synchronized void containerLaunchedOnNode(
@@ -332,8 +298,7 @@ public abstract class AbstractYarnScheduler
 
   @Override
   public SchedulerNodeReport getNodeReport(NodeId nodeId) {
-    N node = nodes.get(nodeId);
-    return node == null ? null : new SchedulerNodeReport(node);
+    return nodeTracker.getNodeReport(nodeId);
   }
 
   @Override
@@ -431,12 +396,13 @@ public abstract class AbstractYarnScheduler
         container));
 
       // recover scheduler node
-      SchedulerNode schedulerNode = nodes.get(nm.getNodeID());
+      SchedulerNode schedulerNode = nodeTracker.getNode(nm.getNodeID());
       schedulerNode.recoverContainer(rmContainer);
 
       // recover queue: update headroom etc.
       Queue queue = schedulerAttempt.getQueue();
-      queue.recoverContainer(clusterResource, schedulerAttempt, rmContainer);
+      queue.recoverContainer(
+          getClusterResource(), schedulerAttempt, rmContainer);
 
       // recover scheduler attempt
       schedulerAttempt.recoverContainer(schedulerNode, rmContainer);
@@ -621,7 +587,7 @@ public abstract class AbstractYarnScheduler
 
   @Override
   public SchedulerNode getSchedulerNode(NodeId nodeId) {
-    return nodes.get(nodeId);
+    return nodeTracker.getNode(nodeId);
   }
 
   @Override
@@ -690,18 +656,12 @@ public abstract class AbstractYarnScheduler
           + " from: " + oldResource + ", to: "
           + newResource);
 
-      nodes.remove(nm.getNodeID());
-      updateMaximumAllocation(node, false);
+      nodeTracker.removeNode(nm.getNodeID());
 
       // update resource to node
       node.setTotalResource(newResource);
 
-      nodes.put(nm.getNodeID(), (N)node);
-      updateMaximumAllocation(node, true);
-
-      // update resource to clusterResource
-      Resources.subtractFrom(clusterResource, oldResource);
-      Resources.addTo(clusterResource, newResource);
+      nodeTracker.addNode((N) node);
     } else {
       // Log resource change
       LOG.warn("Update resource on node: " + node.getNodeName() 
@@ -721,80 +681,8 @@ public abstract class AbstractYarnScheduler
         + " does not support reservations");
   }
 
-  protected void updateMaximumAllocation(SchedulerNode node, boolean add) {
-    Resource totalResource = node.getTotalResource();
-    maxAllocWriteLock.lock();
-    try {
-      if (add) { // added node
-        int nodeMemory = totalResource.getMemory();
-        if (nodeMemory > maxNodeMemory) {
-          maxNodeMemory = nodeMemory;
-          maximumAllocation.setMemory(Math.min(
-              configuredMaximumAllocation.getMemory(), maxNodeMemory));
-        }
-        int nodeVCores = totalResource.getVirtualCores();
-        if (nodeVCores > maxNodeVCores) {
-          maxNodeVCores = nodeVCores;
-          maximumAllocation.setVirtualCores(Math.min(
-              configuredMaximumAllocation.getVirtualCores(), maxNodeVCores));
-        }
-      } else {  // removed node
-        if (maxNodeMemory == totalResource.getMemory()) {
-          maxNodeMemory = -1;
-        }
-        if (maxNodeVCores == totalResource.getVirtualCores()) {
-          maxNodeVCores = -1;
-        }
-        // We only have to iterate through the nodes if the current max memory
-        // or vcores was equal to the removed node's
-        if (maxNodeMemory == -1 || maxNodeVCores == -1) {
-          for (Map.Entry<NodeId, N> nodeEntry : nodes.entrySet()) {
-            int nodeMemory =
-                nodeEntry.getValue().getTotalResource().getMemory();
-            if (nodeMemory > maxNodeMemory) {
-              maxNodeMemory = nodeMemory;
-            }
-            int nodeVCores =
-                nodeEntry.getValue().getTotalResource().getVirtualCores();
-            if (nodeVCores > maxNodeVCores) {
-              maxNodeVCores = nodeVCores;
-            }
-          }
-          if (maxNodeMemory == -1) {  // no nodes
-            maximumAllocation.setMemory(configuredMaximumAllocation.getMemory());
-          } else {
-            maximumAllocation.setMemory(
-                Math.min(configuredMaximumAllocation.getMemory(), maxNodeMemory));
-          }
-          if (maxNodeVCores == -1) {  // no nodes
-            maximumAllocation.setVirtualCores(configuredMaximumAllocation.getVirtualCores());
-          } else {
-            maximumAllocation.setVirtualCores(
-                Math.min(configuredMaximumAllocation.getVirtualCores(), maxNodeVCores));
-          }
-        }
-      }
-    } finally {
-      maxAllocWriteLock.unlock();
-    }
-  }
-
   protected void refreshMaximumAllocation(Resource newMaxAlloc) {
-    maxAllocWriteLock.lock();
-    try {
-      configuredMaximumAllocation = Resources.clone(newMaxAlloc);
-      int maxMemory = newMaxAlloc.getMemory();
-      if (maxNodeMemory != -1) {
-        maxMemory = Math.min(maxMemory, maxNodeMemory);
-      }
-      int maxVcores = newMaxAlloc.getVirtualCores();
-      if (maxNodeVCores != -1) {
-        maxVcores = Math.min(maxVcores, maxNodeVCores);
-      }
-      maximumAllocation = Resources.createResource(maxMemory, maxVcores);
-    } finally {
-      maxAllocWriteLock.unlock();
-    }
+    nodeTracker.setConfiguredMaxAllocation(newMaxAlloc);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hadoop/blob/20d389ce/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/ClusterNodeTracker.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/ClusterNodeTracker.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/ClusterNodeTracker.java
new file mode 100644
index 0000000..34b4267
--- /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/ClusterNodeTracker.java
@@ -0,0 +1,300 @@
+/**
+ * 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;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.yarn.api.records.NodeId;
+import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager;
+import org.apache.hadoop.yarn.util.resource.Resources;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+
+/**
+ * Helper library that:
+ * - tracks the state of all cluster {@link SchedulerNode}s
+ * - provides convenience methods to filter and sort nodes
+ */
+@InterfaceAudience.Private
+public class ClusterNodeTracker<N extends SchedulerNode> {
+  private static final Log LOG = LogFactory.getLog(ClusterNodeTracker.class);
+
+  private ReadWriteLock readWriteLock = new ReentrantReadWriteLock(true);
+  private Lock readLock = readWriteLock.readLock();
+  private Lock writeLock = readWriteLock.writeLock();
+
+  private HashMap<NodeId, N> nodes = new HashMap<>();
+  private Map<String, Integer> nodesPerRack = new HashMap<>();
+
+  private Resource clusterCapacity = Resources.clone(Resources.none());
+  private Resource staleClusterCapacity = null;
+
+  // Max allocation
+  private int maxNodeMemory = -1;
+  private int maxNodeVCores = -1;
+  private Resource configuredMaxAllocation;
+  private boolean forceConfiguredMaxAllocation = true;
+  private long configuredMaxAllocationWaitTime;
+
+  public void addNode(N node) {
+    writeLock.lock();
+    try {
+      nodes.put(node.getNodeID(), node);
+
+      // Update nodes per rack as well
+      String rackName = node.getRackName();
+      Integer numNodes = nodesPerRack.get(rackName);
+      if (numNodes == null) {
+        numNodes = 0;
+      }
+      nodesPerRack.put(rackName, ++numNodes);
+
+      // Update cluster capacity
+      Resources.addTo(clusterCapacity, node.getTotalResource());
+
+      // Update maximumAllocation
+      updateMaxResources(node, true);
+    } finally {
+      writeLock.unlock();
+    }
+  }
+
+  public boolean exists(NodeId nodeId) {
+    readLock.lock();
+    try {
+      return nodes.containsKey(nodeId);
+    } finally {
+      readLock.unlock();
+    }
+  }
+
+  public N getNode(NodeId nodeId) {
+    readLock.lock();
+    try {
+      return nodes.get(nodeId);
+    } finally {
+      readLock.unlock();
+    }
+  }
+
+  public SchedulerNodeReport getNodeReport(NodeId nodeId) {
+    readLock.lock();
+    try {
+      N n = nodes.get(nodeId);
+      return n == null ? null : new SchedulerNodeReport(n);
+    } finally {
+      readLock.unlock();
+    }
+  }
+
+  public int nodeCount() {
+    readLock.lock();
+    try {
+      return nodes.size();
+    } finally {
+      readLock.unlock();
+    }
+  }
+
+  public int nodeCount(String rackName) {
+    readLock.lock();
+    String rName = rackName == null ? "NULL" : rackName;
+    try {
+      Integer nodeCount = nodesPerRack.get(rName);
+      return nodeCount == null ? 0 : nodeCount;
+    } finally {
+      readLock.unlock();
+    }
+  }
+
+  public Resource getClusterCapacity() {
+    readLock.lock();
+    try {
+      if (staleClusterCapacity == null ||
+          !Resources.equals(staleClusterCapacity, clusterCapacity)) {
+        staleClusterCapacity = Resources.clone(clusterCapacity);
+      }
+      return staleClusterCapacity;
+    } finally {
+      readLock.unlock();
+    }
+  }
+
+  public N removeNode(NodeId nodeId) {
+    writeLock.lock();
+    try {
+      N node = nodes.remove(nodeId);
+      if (node == null) {
+        LOG.warn("Attempting to remove a non-existent node " + nodeId);
+        return null;
+      }
+
+      // Update nodes per rack as well
+      String rackName = node.getRackName();
+      Integer numNodes = nodesPerRack.get(rackName);
+      if (numNodes > 0) {
+        nodesPerRack.put(rackName, --numNodes);
+      } else {
+        LOG.error("Attempting to remove node from an empty rack " + rackName);
+      }
+
+      // Update cluster capacity
+      Resources.subtractFrom(clusterCapacity, node.getTotalResource());
+
+      // Update maximumAllocation
+      updateMaxResources(node, false);
+
+      return node;
+    } finally {
+      writeLock.unlock();
+    }
+  }
+
+  public void setConfiguredMaxAllocation(Resource resource) {
+    writeLock.lock();
+    try {
+      configuredMaxAllocation = Resources.clone(resource);
+    } finally {
+      writeLock.unlock();
+    }
+  }
+
+  public void setConfiguredMaxAllocationWaitTime(
+      long configuredMaxAllocationWaitTime) {
+    writeLock.lock();
+    try {
+      this.configuredMaxAllocationWaitTime =
+          configuredMaxAllocationWaitTime;
+    } finally {
+      writeLock.unlock();
+    }
+  }
+
+  public Resource getMaxAllowedAllocation() {
+    readLock.lock();
+    try {
+      if (forceConfiguredMaxAllocation &&
+          System.currentTimeMillis() - ResourceManager.getClusterTimeStamp()
+              > configuredMaxAllocationWaitTime) {
+        forceConfiguredMaxAllocation = false;
+      }
+
+      if (forceConfiguredMaxAllocation
+          || maxNodeMemory == -1 || maxNodeVCores == -1) {
+        return configuredMaxAllocation;
+      }
+
+      return Resources.createResource(
+          Math.min(configuredMaxAllocation.getMemory(), maxNodeMemory),
+          Math.min(configuredMaxAllocation.getVirtualCores(), maxNodeVCores)
+      );
+    } finally {
+      readLock.unlock();
+    }
+  }
+
+  private void updateMaxResources(SchedulerNode node, boolean add) {
+    Resource totalResource = node.getTotalResource();
+    writeLock.lock();
+    try {
+      if (add) { // added node
+        int nodeMemory = totalResource.getMemory();
+        if (nodeMemory > maxNodeMemory) {
+          maxNodeMemory = nodeMemory;
+        }
+        int nodeVCores = totalResource.getVirtualCores();
+        if (nodeVCores > maxNodeVCores) {
+          maxNodeVCores = nodeVCores;
+        }
+      } else {  // removed node
+        if (maxNodeMemory == totalResource.getMemory()) {
+          maxNodeMemory = -1;
+        }
+        if (maxNodeVCores == totalResource.getVirtualCores()) {
+          maxNodeVCores = -1;
+        }
+        // We only have to iterate through the nodes if the current max memory
+        // or vcores was equal to the removed node's
+        if (maxNodeMemory == -1 || maxNodeVCores == -1) {
+          // Treat it like an empty cluster and add nodes
+          for (N n : nodes.values()) {
+            updateMaxResources(n, true);
+          }
+        }
+      }
+    } finally {
+      writeLock.unlock();
+    }
+  }
+
+  public List<N> getAllNodes() {
+    return getNodes(null);
+  }
+
+  /**
+   * Convenience method to filter nodes based on a condition.
+   */
+  public List<N> getNodes(NodeFilter nodeFilter) {
+    List<N> nodeList = new ArrayList<>();
+    readLock.lock();
+    try {
+      if (nodeFilter == null) {
+        nodeList.addAll(nodes.values());
+      } else {
+        for (N node : nodes.values()) {
+          if (nodeFilter.accept(node)) {
+            nodeList.add(node);
+          }
+        }
+      }
+    } finally {
+      readLock.unlock();
+    }
+    return nodeList;
+  }
+
+  /**
+   * Convenience method to sort nodes.
+   *
+   * Note that the sort is performed without holding a lock. We are sorting
+   * here instead of on the caller to allow for future optimizations (e.g.
+   * sort once every x milliseconds).
+   */
+  public List<N> sortedNodeList(Comparator<N> comparator) {
+    List<N> sortedList = null;
+    readLock.lock();
+    try {
+      sortedList = new ArrayList(nodes.values());
+    } finally {
+      readLock.unlock();
+    }
+    Collections.sort(sortedList, comparator);
+    return sortedList;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/20d389ce/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/NodeFilter.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/NodeFilter.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/NodeFilter.java
new file mode 100644
index 0000000..7b3e7a2
--- /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/NodeFilter.java
@@ -0,0 +1,33 @@
+/**
+ * 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;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+
+/**
+ * Convenience way to filter nodes based on a criteria. To be used in
+ * conjunction with {@link ClusterNodeTracker}
+ */
+@InterfaceAudience.Private
+public interface NodeFilter {
+
+  /**
+   * Criteria to accept node in the filtered list.
+   */
+  boolean accept(SchedulerNode node);
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/20d389ce/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 6a1091d..735306a 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
@@ -34,7 +34,6 @@ import java.util.Random;
 import java.util.Set;
 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;
@@ -219,8 +218,6 @@ public class CapacityScheduler extends
 
   private Map<String, CSQueue> queues = new ConcurrentHashMap<String, CSQueue>();
 
-  private AtomicInteger numNodeManagers = new AtomicInteger(0);
-
   private ResourceCalculator calculator;
   private boolean usePortForNodeName;
 
@@ -280,7 +277,7 @@ public class CapacityScheduler extends
 
   @Override
   public int getNumClusterNodes() {
-    return numNodeManagers.get();
+    return nodeTracker.nodeCount();
   }
 
   @Override
@@ -387,7 +384,7 @@ public class CapacityScheduler extends
   static void schedule(CapacityScheduler cs) {
     // First randomize the start point
     int current = 0;
-    Collection<FiCaSchedulerNode> nodes = cs.getAllNodes().values();
+    Collection<FiCaSchedulerNode> nodes = cs.nodeTracker.getAllNodes();
     int start = random.nextInt(nodes.size());
     for (FiCaSchedulerNode node : nodes) {
       if (current++ >= start) {
@@ -524,10 +521,11 @@ public class CapacityScheduler extends
     addNewQueues(queues, newQueues);
     
     // Re-configure queues
-    root.reinitialize(newRoot, clusterResource);
+    root.reinitialize(newRoot, getClusterResource());
     updatePlacementRules();
 
     // Re-calculate headroom for active applications
+    Resource clusterResource = getClusterResource();
     root.updateClusterResource(clusterResource, new ResourceLimits(
         clusterResource));
 
@@ -995,7 +993,7 @@ public class CapacityScheduler extends
       
 
       allocation = application.getAllocation(getResourceCalculator(),
-                   clusterResource, getMinimumResourceCapability());
+          getClusterResource(), getMinimumResourceCapability());
     }
 
     if (updateDemandForQueue != null && !application
@@ -1036,7 +1034,8 @@ public class CapacityScheduler extends
 
   private synchronized void nodeUpdate(RMNode nm) {
     if (LOG.isDebugEnabled()) {
-      LOG.debug("nodeUpdate: " + nm + " clusterResources: " + clusterResource);
+      LOG.debug("nodeUpdate: " + nm +
+          " clusterResources: " + getClusterResource());
     }
 
     Resource releaseResources = Resource.newInstance(0, 0);
@@ -1119,6 +1118,7 @@ public class CapacityScheduler extends
   private synchronized void updateNodeAndQueueResource(RMNode nm, 
       ResourceOption resourceOption) {
     updateNodeResource(nm, resourceOption);
+    Resource clusterResource = getClusterResource();
     root.updateClusterResource(clusterResource, new ResourceLimits(
         clusterResource));
   }
@@ -1128,7 +1128,7 @@ public class CapacityScheduler extends
    */
   private synchronized void updateLabelsOnNode(NodeId nodeId,
       Set<String> newLabels) {
-    FiCaSchedulerNode node = nodes.get(nodeId);
+    FiCaSchedulerNode node = nodeTracker.getNode(nodeId);
     if (null == node) {
       return;
     }
@@ -1230,12 +1230,12 @@ public class CapacityScheduler extends
       LeafQueue queue = ((LeafQueue) reservedApplication.getQueue());
       assignment =
           queue.assignContainers(
-              clusterResource,
+              getClusterResource(),
               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, getClusterResource())),
               SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
       if (assignment.isFulfilledReservation()) {
         CSAssignment tmp =
@@ -1261,14 +1261,14 @@ public class CapacityScheduler extends
         }
 
         assignment = root.assignContainers(
-            clusterResource,
+            getClusterResource(),
             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, getClusterResource())),
             SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
-        if (Resources.greaterThan(calculator, clusterResource,
+        if (Resources.greaterThan(calculator, getClusterResource(),
             assignment.getResource(), Resources.none())) {
           updateSchedulerHealth(lastNodeUpdateTime, node, assignment);
           return;
@@ -1294,12 +1294,12 @@ public class CapacityScheduler extends
         
         // Try to use NON_EXCLUSIVE
         assignment = root.assignContainers(
-            clusterResource,
+            getClusterResource(),
             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, getClusterResource())),
             SchedulingMode.IGNORE_PARTITION_EXCLUSIVITY);
         updateSchedulerHealth(lastNodeUpdateTime, node, assignment);
       }
@@ -1451,24 +1451,22 @@ public class CapacityScheduler extends
   private synchronized void addNode(RMNode nodeManager) {
     FiCaSchedulerNode schedulerNode = new FiCaSchedulerNode(nodeManager,
         usePortForNodeName, nodeManager.getNodeLabels());
-    this.nodes.put(nodeManager.getNodeID(), schedulerNode);
-    Resources.addTo(clusterResource, schedulerNode.getTotalResource());
+    nodeTracker.addNode(schedulerNode);
 
     // update this node to node label manager
     if (labelManager != null) {
       labelManager.activateNode(nodeManager.getNodeID(),
           schedulerNode.getTotalResource());
     }
-    
+
+    Resource clusterResource = getClusterResource();
     root.updateClusterResource(clusterResource, new ResourceLimits(
         clusterResource));
-    int numNodes = numNodeManagers.incrementAndGet();
-    updateMaximumAllocation(schedulerNode, true);
-    
+
     LOG.info("Added node " + nodeManager.getNodeAddress() + 
         " clusterResource: " + clusterResource);
 
-    if (scheduleAsynchronously && numNodes == 1) {
+    if (scheduleAsynchronously && getNumClusterNodes() == 1) {
       asyncSchedulerThread.beginSchedule();
     }
   }
@@ -1478,20 +1476,14 @@ public class CapacityScheduler extends
     if (labelManager != null) {
       labelManager.deactivateNode(nodeInfo.getNodeID());
     }
-    
-    FiCaSchedulerNode node = nodes.get(nodeInfo.getNodeID());
+
+    NodeId nodeId = nodeInfo.getNodeID();
+    FiCaSchedulerNode node = nodeTracker.getNode(nodeId);
     if (node == null) {
+      LOG.error("Attempting to remove non-existent node " + nodeId);
       return;
     }
-    Resources.subtractFrom(clusterResource, node.getTotalResource());
-    root.updateClusterResource(clusterResource, new ResourceLimits(
-        clusterResource));
-    int numNodes = numNodeManagers.decrementAndGet();
 
-    if (scheduleAsynchronously && numNodes == 0) {
-      asyncSchedulerThread.suspendSchedule();
-    }
-    
     // Remove running containers
     List<RMContainer> runningContainers = node.getRunningContainers();
     for (RMContainer container : runningContainers) {
@@ -1512,11 +1504,18 @@ public class CapacityScheduler extends
           RMContainerEventType.KILL);
     }
 
-    this.nodes.remove(nodeInfo.getNodeID());
-    updateMaximumAllocation(node, false);
+    nodeTracker.removeNode(nodeId);
+    Resource clusterResource = getClusterResource();
+    root.updateClusterResource(clusterResource, new ResourceLimits(
+        clusterResource));
+    int numNodes = nodeTracker.nodeCount();
+
+    if (scheduleAsynchronously && numNodes == 0) {
+      asyncSchedulerThread.suspendSchedule();
+    }
 
     LOG.info("Removed node " + nodeInfo.getNodeAddress() + 
-        " clusterResource: " + clusterResource);
+        " clusterResource: " + getClusterResource());
   }
 
   private void rollbackContainerResource(
@@ -1568,7 +1567,7 @@ public class CapacityScheduler extends
     
     // Inform the queue
     LeafQueue queue = (LeafQueue)application.getQueue();
-    queue.completedContainer(clusterResource, application, node, 
+    queue.completedContainer(getClusterResource(), application, node,
         rmContainer, containerStatus, event, null, true);
 
     if (containerStatus.getExitStatus() == ContainerExitStatus.PREEMPTED) {
@@ -1594,7 +1593,7 @@ public class CapacityScheduler extends
     FiCaSchedulerApp app = (FiCaSchedulerApp)attempt;
     LeafQueue queue = (LeafQueue) attempt.getQueue();
     try {
-      queue.decreaseContainer(clusterResource, decreaseRequest, app);
+      queue.decreaseContainer(getClusterResource(), decreaseRequest, app);
       // Notify RMNode that the container can be pulled by NodeManager in the
       // next heartbeat
       this.rmContext.getDispatcher().getEventHandler()
@@ -1617,14 +1616,9 @@ public class CapacityScheduler extends
   
   @Lock(Lock.NoLock.class)
   public FiCaSchedulerNode getNode(NodeId nodeId) {
-    return nodes.get(nodeId);
+    return nodeTracker.getNode(nodeId);
   }
   
-  @Lock(Lock.NoLock.class)
-  Map<NodeId, FiCaSchedulerNode> getAllNodes() {
-    return nodes;
-  }
-
   @Override
   @Lock(Lock.NoLock.class)
   public void recover(RMState state) throws Exception {
@@ -1869,9 +1863,9 @@ public class CapacityScheduler extends
     }
     // Move all live containers
     for (RMContainer rmContainer : app.getLiveContainers()) {
-      source.detachContainer(clusterResource, app, rmContainer);
+      source.detachContainer(getClusterResource(), app, rmContainer);
       // attach the Container to another queue
-      dest.attachContainer(clusterResource, app, rmContainer);
+      dest.attachContainer(getClusterResource(), app, rmContainer);
     }
     // Detach the application..
     source.finishApplicationAttempt(app, sourceQueueName);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/20d389ce/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSAppAttempt.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/fair/FSAppAttempt.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSAppAttempt.java
index f1cefad..e426da6 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSAppAttempt.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSAppAttempt.java
@@ -86,7 +86,7 @@ public class FSAppAttempt extends SchedulerApplicationAttempt
   // Key = RackName, Value = Set of Nodes reserved by app on rack
   private Map<String, Set<String>> reservations = new HashMap<>();
 
-  private List<NodeId> blacklistNodeIds = new ArrayList<NodeId>();
+  private List<FSSchedulerNode> blacklistNodeIds = new ArrayList<>();
   /**
    * Delay scheduling: We often want to prioritize scheduling of node-local
    * containers over rack-local or off-switch containers. To achieve this
@@ -185,14 +185,11 @@ public class FSAppAttempt extends SchedulerApplicationAttempt
       Resource availableResources) {
     if (appSchedulingInfo.getAndResetBlacklistChanged()) {
       blacklistNodeIds.clear();
-      scheduler.addBlacklistedNodeIdsToList(this, blacklistNodeIds);
+      blacklistNodeIds.addAll(scheduler.getBlacklistedNodes(this));
     }
-    for (NodeId nodeId: blacklistNodeIds) {
-      SchedulerNode node = scheduler.getSchedulerNode(nodeId);
-      if (node != null) {
-        Resources.subtractFrom(availableResources,
-            node.getUnallocatedResource());
-      }
+    for (FSSchedulerNode node: blacklistNodeIds) {
+      Resources.subtractFrom(availableResources,
+          node.getUnallocatedResource());
     }
     if (availableResources.getMemory() < 0) {
       availableResources.setMemory(0);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/20d389ce/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FairScheduler.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/fair/FairScheduler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FairScheduler.java
index 917fc8a..ba90e21 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FairScheduler.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FairScheduler.java
@@ -20,13 +20,11 @@ package org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair;
 
 import java.io.IOException;
 import java.util.ArrayList;
-import java.util.Collections;
 import java.util.Comparator;
 import java.util.EnumSet;
 import java.util.HashSet;
 import java.util.Iterator;
 import java.util.List;
-import java.util.Map;
 import java.util.Set;
 import java.util.concurrent.ConcurrentHashMap;
 
@@ -186,14 +184,11 @@ public class FairScheduler extends
   private float reservableNodesRatio; // percentage of available nodes
                                       // an app can be reserved on
 
-  // Count of number of nodes per rack
-  private Map<String, Integer> nodesPerRack = new ConcurrentHashMap<>();
-
   protected boolean sizeBasedWeight; // Give larger weights to larger jobs
   protected WeightAdjuster weightAdjuster; // Can be null for no weight adjuster
   protected boolean continuousSchedulingEnabled; // Continuous Scheduling enabled or not
   protected int continuousSchedulingSleepMs; // Sleep time for each pass in continuous scheduling
-  private Comparator<NodeId> nodeAvailableResourceComparator =
+  private Comparator<FSSchedulerNode> nodeAvailableResourceComparator =
           new NodeAvailableResourceComparator(); // Node available resource comparator
   protected double nodeLocalityThreshold; // Cluster threshold for node locality
   protected double rackLocalityThreshold; // Cluster threshold for rack locality
@@ -225,8 +220,8 @@ public class FairScheduler extends
 
   public boolean isAtLeastReservationThreshold(
       ResourceCalculator resourceCalculator, Resource resource) {
-    return Resources.greaterThanOrEqual(
-        resourceCalculator, clusterResource, resource, reservationThreshold);
+    return Resources.greaterThanOrEqual(resourceCalculator,
+        getClusterResource(), resource, reservationThreshold);
   }
 
   private void validateConf(Configuration conf) {
@@ -272,11 +267,7 @@ public class FairScheduler extends
   }
 
   public int getNumNodesInRack(String rackName) {
-    String rName = rackName == null ? "NULL" : rackName;
-    if (nodesPerRack.containsKey(rName)) {
-      return nodesPerRack.get(rName);
-    }
-    return 0;
+    return nodeTracker.nodeCount(rackName);
   }
 
   public QueueManager getQueueManager() {
@@ -352,6 +343,7 @@ public class FairScheduler extends
     // Recursively update demands for all queues
     rootQueue.updateDemand();
 
+    Resource clusterResource = getClusterResource();
     rootQueue.setFairShare(clusterResource);
     // Recursively compute fair shares for all queues
     // and update metrics
@@ -526,6 +518,7 @@ public class FairScheduler extends
     Resource resDueToMinShare = Resources.none();
     Resource resDueToFairShare = Resources.none();
     ResourceCalculator calc = sched.getPolicy().getResourceCalculator();
+    Resource clusterResource = getClusterResource();
     if (curTime - sched.getLastTimeAtMinShare() > minShareTimeout) {
       Resource target = Resources.componentwiseMin(
           sched.getMinShare(), sched.getDemand());
@@ -577,7 +570,7 @@ public class FairScheduler extends
   }
 
   private FSSchedulerNode getFSSchedulerNode(NodeId nodeId) {
-    return nodes.get(nodeId);
+    return nodeTracker.getNode(nodeId);
   }
 
   public double getNodeLocalityThreshold() {
@@ -882,18 +875,11 @@ public class FairScheduler extends
   private synchronized void addNode(List<NMContainerStatus> containerReports,
       RMNode node) {
     FSSchedulerNode schedulerNode = new FSSchedulerNode(node, usePortForNodeName);
-    nodes.put(node.getNodeID(), schedulerNode);
-    String rackName = node.getRackName() == null ? "NULL" : node.getRackName();
-    if (nodesPerRack.containsKey(rackName)) {
-      nodesPerRack.put(rackName, nodesPerRack.get(rackName) + 1);
-    } else {
-      nodesPerRack.put(rackName, 1);
-    }
-    Resources.addTo(clusterResource, schedulerNode.getTotalResource());
-    updateMaximumAllocation(schedulerNode, true);
+    nodeTracker.addNode(schedulerNode);
 
     triggerUpdate();
 
+    Resource clusterResource = getClusterResource();
     queueMgr.getRootQueue().setSteadyFairShare(clusterResource);
     queueMgr.getRootQueue().recomputeSteadyShares();
     LOG.info("Added node " + node.getNodeAddress() +
@@ -904,15 +890,12 @@ public class FairScheduler extends
   }
 
   private synchronized void removeNode(RMNode rmNode) {
-    FSSchedulerNode node = getFSSchedulerNode(rmNode.getNodeID());
-    // This can occur when an UNHEALTHY node reconnects
+    NodeId nodeId = rmNode.getNodeID();
+    FSSchedulerNode node = nodeTracker.getNode(nodeId);
     if (node == null) {
+      LOG.error("Attempting to remove non-existent node " + nodeId);
       return;
     }
-    Resources.subtractFrom(clusterResource, node.getTotalResource());
-    updateRootQueueMetrics();
-
-    triggerUpdate();
 
     // Remove running containers
     List<RMContainer> runningContainers = node.getRunningContainers();
@@ -934,18 +917,13 @@ public class FairScheduler extends
           RMContainerEventType.KILL);
     }
 
-    nodes.remove(rmNode.getNodeID());
-    String rackName = node.getRackName() == null ? "NULL" : node.getRackName();
-    if (nodesPerRack.containsKey(rackName)
-            && (nodesPerRack.get(rackName) > 0)) {
-      nodesPerRack.put(rackName, nodesPerRack.get(rackName) - 1);
-    } else {
-      LOG.error("Node [" + rmNode.getNodeAddress() + "] being removed from" +
-              " unknown rack [" + rackName + "] !!");
-    }
+    nodeTracker.removeNode(nodeId);
+    Resource clusterResource = getClusterResource();
     queueMgr.getRootQueue().setSteadyFairShare(clusterResource);
     queueMgr.getRootQueue().recomputeSteadyShares();
-    updateMaximumAllocation(node, false);
+    updateRootQueueMetrics();
+    triggerUpdate();
+
     LOG.info("Removed node " + rmNode.getNodeAddress() +
         " cluster capacity: " + clusterResource);
   }
@@ -967,7 +945,7 @@ public class FairScheduler extends
 
     // Sanity check
     SchedulerUtils.normalizeRequests(ask, DOMINANT_RESOURCE_CALCULATOR,
-        clusterResource, minimumAllocation, getMaximumResourceCapability(),
+        getClusterResource(), minimumAllocation, getMaximumResourceCapability(),
         incrAllocation);
 
     // Record container allocation start time
@@ -1034,7 +1012,8 @@ public class FairScheduler extends
   private synchronized void nodeUpdate(RMNode nm) {
     long start = getClock().getTime();
     if (LOG.isDebugEnabled()) {
-      LOG.debug("nodeUpdate: " + nm + " cluster capacity: " + clusterResource);
+      LOG.debug("nodeUpdate: " + nm +
+          " cluster capacity: " + getClusterResource());
     }
     eventLog.log("HEARTBEAT", nm.getHostName());
     FSSchedulerNode node = getFSSchedulerNode(nm.getNodeID());
@@ -1091,20 +1070,13 @@ public class FairScheduler extends
 
   void continuousSchedulingAttempt() throws InterruptedException {
     long start = getClock().getTime();
-    List<NodeId> nodeIdList = new ArrayList<NodeId>(nodes.keySet());
-    // Sort the nodes by space available on them, so that we offer
-    // containers on emptier nodes first, facilitating an even spread. This
-    // requires holding the scheduler lock, so that the space available on a
-    // node doesn't change during the sort.
-    synchronized (this) {
-      Collections.sort(nodeIdList, nodeAvailableResourceComparator);
-    }
+    List<FSSchedulerNode> nodeIdList =
+        nodeTracker.sortedNodeList(nodeAvailableResourceComparator);
 
     // iterate all nodes
-    for (NodeId nodeId : nodeIdList) {
-      FSSchedulerNode node = getFSSchedulerNode(nodeId);
+    for (FSSchedulerNode node : nodeIdList) {
       try {
-        if (node != null && Resources.fitsIn(minimumAllocation,
+        if (Resources.fitsIn(minimumAllocation,
             node.getUnallocatedResource())) {
           attemptScheduling(node);
         }
@@ -1126,19 +1098,14 @@ public class FairScheduler extends
   }
 
   /** Sort nodes by available resource */
-  private class NodeAvailableResourceComparator implements Comparator<NodeId> {
+  private class NodeAvailableResourceComparator
+      implements Comparator<FSSchedulerNode> {
 
     @Override
-    public int compare(NodeId n1, NodeId n2) {
-      if (!nodes.containsKey(n1)) {
-        return 1;
-      }
-      if (!nodes.containsKey(n2)) {
-        return -1;
-      }
-      return RESOURCE_CALCULATOR.compare(clusterResource,
-              nodes.get(n2).getUnallocatedResource(),
-              nodes.get(n1).getUnallocatedResource());
+    public int compare(FSSchedulerNode n1, FSSchedulerNode n2) {
+      return RESOURCE_CALCULATOR.compare(getClusterResource(),
+          n2.getUnallocatedResource(),
+          n1.getUnallocatedResource());
     }
   }
 
@@ -1150,7 +1117,7 @@ public class FairScheduler extends
     }
 
     final NodeId nodeID = node.getNodeID();
-    if (!nodes.containsKey(nodeID)) {
+    if (!nodeTracker.exists(nodeID)) {
       // The node might have just been removed while this thread was waiting
       // on the synchronized lock before it entered this synchronized method
       LOG.info("Skipping scheduling as the node " + nodeID +
@@ -1203,7 +1170,7 @@ public class FairScheduler extends
   private void updateRootQueueMetrics() {
     rootMetrics.setAvailableResourcesToQueue(
         Resources.subtract(
-            clusterResource, rootMetrics.getAllocatedResources()));
+            getClusterResource(), rootMetrics.getAllocatedResources()));
   }
 
   /**
@@ -1214,6 +1181,7 @@ public class FairScheduler extends
    */
   private boolean shouldAttemptPreemption() {
     if (preemptionEnabled) {
+      Resource clusterResource = getClusterResource();
       return (preemptionUtilizationThreshold < Math.max(
           (float) rootMetrics.getAllocatedMB() / clusterResource.getMemory(),
           (float) rootMetrics.getAllocatedVirtualCores() /
@@ -1547,7 +1515,7 @@ public class FairScheduler extends
 
   @Override
   public int getNumClusterNodes() {
-    return nodes.size();
+    return nodeTracker.nodeCount();
   }
 
   @Override
@@ -1577,7 +1545,7 @@ public class FairScheduler extends
       // if it does not already exist, so it can be displayed on the web UI.
       synchronized (FairScheduler.this) {
         allocConf = queueInfo;
-        allocConf.getDefaultSchedulingPolicy().initialize(clusterResource);
+        allocConf.getDefaultSchedulingPolicy().initialize(getClusterResource());
         queueMgr.updateAllocationConfiguration(allocConf);
         maxRunningEnforcer.updateRunnabilityOnReload();
       }
@@ -1721,7 +1689,7 @@ public class FairScheduler extends
       ResourceOption resourceOption) {
     super.updateNodeResource(nm, resourceOption);
     updateRootQueueMetrics();
-    queueMgr.getRootQueue().setSteadyFairShare(clusterResource);
+    queueMgr.getRootQueue().setSteadyFairShare(getClusterResource());
     queueMgr.getRootQueue().recomputeSteadyShares();
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/20d389ce/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fifo/FifoScheduler.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/fifo/FifoScheduler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fifo/FifoScheduler.java
index 147c3f3..cf12501 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fifo/FifoScheduler.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fifo/FifoScheduler.java
@@ -142,6 +142,7 @@ public class FifoScheduler extends
       QueueInfo queueInfo = recordFactory.newRecordInstance(QueueInfo.class);
       queueInfo.setQueueName(DEFAULT_QUEUE.getQueueName());
       queueInfo.setCapacity(1.0f);
+      Resource clusterResource = getClusterResource();
       if (clusterResource.getMemory() == 0) {
         queueInfo.setCurrentCapacity(0.0f);
       } else {
@@ -297,7 +298,7 @@ public class FifoScheduler extends
 
   @Override
   public int getNumClusterNodes() {
-    return nodes.size();
+    return nodeTracker.nodeCount();
   }
 
   @Override
@@ -327,7 +328,8 @@ public class FifoScheduler extends
 
     // Sanity check
     SchedulerUtils.normalizeRequests(ask, resourceCalculator, 
-        clusterResource, minimumAllocation, getMaximumResourceCapability());
+        getClusterResource(), minimumAllocation,
+        getMaximumResourceCapability());
 
     // Release containers
     releaseContainers(release, application);
@@ -377,7 +379,7 @@ public class FifoScheduler extends
   }
 
   private FiCaSchedulerNode getNode(NodeId nodeId) {
-    return nodes.get(nodeId);
+    return nodeTracker.getNode(nodeId);
   }
 
   @VisibleForTesting
@@ -526,7 +528,7 @@ public class FifoScheduler extends
       application.showRequests();
 
       // Done
-      if (Resources.lessThan(resourceCalculator, clusterResource,
+      if (Resources.lessThan(resourceCalculator, getClusterResource(),
               node.getUnallocatedResource(), minimumAllocation)) {
         break;
       }
@@ -764,7 +766,7 @@ public class FifoScheduler extends
       return;
     }
 
-    if (Resources.greaterThanOrEqual(resourceCalculator, clusterResource,
+    if (Resources.greaterThanOrEqual(resourceCalculator, getClusterResource(),
             node.getUnallocatedResource(), minimumAllocation)) {
       LOG.debug("Node heartbeat " + rmNode.getNodeID() + 
           " available resource = " + node.getUnallocatedResource());
@@ -783,13 +785,13 @@ public class FifoScheduler extends
   }
 
   private void updateAppHeadRoom(SchedulerApplicationAttempt schedulerAttempt) {
-    schedulerAttempt.setHeadroom(Resources.subtract(clusterResource,
+    schedulerAttempt.setHeadroom(Resources.subtract(getClusterResource(),
       usedResource));
   }
 
   private void updateAvailableResourcesMetrics() {
-    metrics.setAvailableResourcesToQueue(Resources.subtract(clusterResource,
-      usedResource));
+    metrics.setAvailableResourcesToQueue(
+        Resources.subtract(getClusterResource(), usedResource));
   }
 
   @Override
@@ -925,7 +927,7 @@ public class FifoScheduler extends
   private Resource usedResource = recordFactory.newRecordInstance(Resource.class);
 
   private synchronized void removeNode(RMNode nodeInfo) {
-    FiCaSchedulerNode node = getNode(nodeInfo.getNodeID());
+    FiCaSchedulerNode node = nodeTracker.getNode(nodeInfo.getNodeID());
     if (node == null) {
       return;
     }
@@ -937,13 +939,7 @@ public class FifoScheduler extends
               SchedulerUtils.LOST_CONTAINER),
               RMContainerEventType.KILL);
     }
-    
-    //Remove the node
-    this.nodes.remove(nodeInfo.getNodeID());
-    updateMaximumAllocation(node, false);
-    
-    // Update cluster metrics
-    Resources.subtractFrom(clusterResource, node.getTotalResource());
+    nodeTracker.removeNode(nodeInfo.getNodeID());
   }
 
   @Override
@@ -965,9 +961,7 @@ public class FifoScheduler extends
   private synchronized void addNode(RMNode nodeManager) {
     FiCaSchedulerNode schedulerNode = new FiCaSchedulerNode(nodeManager,
         usePortForNodeName);
-    this.nodes.put(nodeManager.getNodeID(), schedulerNode);
-    Resources.addTo(clusterResource, schedulerNode.getTotalResource());
-    updateMaximumAllocation(schedulerNode, true);
+    nodeTracker.addNode(schedulerNode);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hadoop/blob/20d389ce/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/TestAbstractYarnScheduler.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/TestAbstractYarnScheduler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/TestAbstractYarnScheduler.java
index e7ba58d..81c8fe6 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/TestAbstractYarnScheduler.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/TestAbstractYarnScheduler.java
@@ -300,22 +300,16 @@ public class TestAbstractYarnScheduler extends ParameterizedSchedulerTestBase {
 
       verifyMaximumResourceCapability(configuredMaximumResource, scheduler);
 
-      scheduler.nodes = new HashMap<NodeId, SchedulerNode>();
-
-      scheduler.nodes.put(mockNode1.getNodeID(), mockNode1);
-      scheduler.updateMaximumAllocation(mockNode1, true);
+      scheduler.nodeTracker.addNode(mockNode1);
       verifyMaximumResourceCapability(fullResource1, scheduler);
 
-      scheduler.nodes.put(mockNode2.getNodeID(), mockNode2);
-      scheduler.updateMaximumAllocation(mockNode2, true);
+      scheduler.nodeTracker.addNode(mockNode2);
       verifyMaximumResourceCapability(fullResource2, scheduler);
 
-      scheduler.nodes.remove(mockNode2.getNodeID());
-      scheduler.updateMaximumAllocation(mockNode2, false);
+      scheduler.nodeTracker.removeNode(mockNode2.getNodeID());
       verifyMaximumResourceCapability(fullResource1, scheduler);
 
-      scheduler.nodes.remove(mockNode1.getNodeID());
-      scheduler.updateMaximumAllocation(mockNode1, false);
+      scheduler.nodeTracker.removeNode(mockNode1.getNodeID());
       verifyMaximumResourceCapability(configuredMaximumResource, scheduler);
     } finally {
       rm.stop();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/20d389ce/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 9047138..2ef5e39 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
@@ -183,6 +183,7 @@ public class TestReservations {
   }
 
   @Test
+  @SuppressWarnings("unchecked")
   public void testReservation() throws Exception {
     // Test that we now unreserve and use a node that has space
 
@@ -231,9 +232,9 @@ public class TestReservations {
     when(csContext.getNode(node_1.getNodeID())).thenReturn(node_1);
     when(csContext.getNode(node_2.getNodeID())).thenReturn(node_2);
 
-    cs.getAllNodes().put(node_0.getNodeID(), node_0);
-    cs.getAllNodes().put(node_1.getNodeID(), node_1);
-    cs.getAllNodes().put(node_2.getNodeID(), node_2);
+    cs.getNodeTracker().addNode(node_0);
+    cs.getNodeTracker().addNode(node_1);
+    cs.getNodeTracker().addNode(node_2);
 
     final int numNodes = 3;
     Resource clusterResource = Resources.createResource(numNodes * (8 * GB));
@@ -346,6 +347,7 @@ public class TestReservations {
   // Test that hitting a reservation limit and needing to unreserve
   // does not affect assigning containers for other users
   @Test
+  @SuppressWarnings("unchecked")
   public void testReservationLimitOtherUsers() throws Exception {
     CapacitySchedulerConfiguration csConf = new CapacitySchedulerConfiguration();
     setup(csConf, true);
@@ -395,9 +397,9 @@ public class TestReservations {
     when(csContext.getNode(node_1.getNodeID())).thenReturn(node_1);
     when(csContext.getNode(node_2.getNodeID())).thenReturn(node_2);
 
-    cs.getAllNodes().put(node_0.getNodeID(), node_0);
-    cs.getAllNodes().put(node_1.getNodeID(), node_1);
-    cs.getAllNodes().put(node_2.getNodeID(), node_2);
+    cs.getNodeTracker().addNode(node_0);
+    cs.getNodeTracker().addNode(node_1);
+    cs.getNodeTracker().addNode(node_2);
 
     final int numNodes = 3;
     Resource clusterResource = Resources.createResource(numNodes * (8 * GB));
@@ -641,6 +643,7 @@ public class TestReservations {
   }
 
   @Test
+  @SuppressWarnings("unchecked")
   public void testAssignContainersNeedToUnreserve() throws Exception {
     // Test that we now unreserve and use a node that has space
     Logger rootLogger = LogManager.getRootLogger();
@@ -684,8 +687,8 @@ public class TestReservations {
     FiCaSchedulerNode node_1 = TestUtils.getMockNode(host_1, DEFAULT_RACK, 0,
         8 * GB);
 
-    cs.getAllNodes().put(node_0.getNodeID(), node_0);
-    cs.getAllNodes().put(node_1.getNodeID(), node_1);
+    cs.getNodeTracker().addNode(node_0);
+    cs.getNodeTracker().addNode(node_1);
 
     when(csContext.getNode(node_0.getNodeID())).thenReturn(node_0);
     when(csContext.getNode(node_1.getNodeID())).thenReturn(node_1);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/20d389ce/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFairScheduler.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/fair/TestFairScheduler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFairScheduler.java
index 8d7c22e..1add193 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFairScheduler.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFairScheduler.java
@@ -75,12 +75,10 @@ import org.apache.hadoop.yarn.event.Event;
 import org.apache.hadoop.yarn.event.EventHandler;
 import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
-import org.apache.hadoop.yarn.server.resourcemanager.Application;
 import org.apache.hadoop.yarn.server.resourcemanager.ApplicationMasterService;
 import org.apache.hadoop.yarn.server.resourcemanager.MockNodes;
 import org.apache.hadoop.yarn.server.resourcemanager.MockRM;
 import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
-import org.apache.hadoop.yarn.server.resourcemanager.Task;
 import org.apache.hadoop.yarn.server.resourcemanager.resource.ResourceType;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.MockRMApp;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
@@ -108,7 +106,6 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeUpdateS
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.QueuePlacementRule.Default;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.policies.DominantResourceFairnessPolicy;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.policies.FifoPolicy;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fifo.FifoScheduler;
 import org.apache.hadoop.yarn.server.utils.BuilderUtils;
 import org.apache.hadoop.yarn.util.ControlledClock;
 import org.apache.hadoop.yarn.util.resource.Resources;
@@ -2751,8 +2748,8 @@ public class TestFairScheduler extends FairSchedulerTestBase {
 
     RMNode node1 = MockNodes.newNodeInfo(1, Resources.createResource(1024), 1, "127.0.0.1");
     RMNode node2 = MockNodes.newNodeInfo(1, Resources.createResource(1024), 2, "127.0.0.2");
-    NodeAddedSchedulerEvent nodeEvent2 = new NodeAddedSchedulerEvent(node1);
-    scheduler.handle(nodeEvent2);
+    NodeAddedSchedulerEvent nodeEvent1 = new NodeAddedSchedulerEvent(node1);
+    scheduler.handle(nodeEvent1);
 
     ApplicationAttemptId attId = createSchedulingRequest(1024, "queue1",
         "user1", 0);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/20d389ce/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 9bfc283..44877fb 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
@@ -306,12 +306,7 @@ public class TestFifoScheduler {
     nmTokenSecretManager.rollMasterKey();
     RMApplicationHistoryWriter writer = mock(RMApplicationHistoryWriter.class);
     
-    FifoScheduler scheduler = new FifoScheduler(){
-      @SuppressWarnings("unused")
-      public Map<NodeId, FiCaSchedulerNode> getNodes(){
-        return nodes;
-      }
-    };
+    FifoScheduler scheduler = new FifoScheduler();
     RMContext rmContext = new RMContextImpl(dispatcher, null, null, null, null,
         null, containerTokenSecretManager, nmTokenSecretManager, null, scheduler);
     rmContext.setSystemMetricsPublisher(mock(SystemMetricsPublisher.class));
@@ -331,11 +326,7 @@ public class TestFifoScheduler {
     NodeAddedSchedulerEvent nodeEvent1 = new NodeAddedSchedulerEvent(node0);
     scheduler.handle(nodeEvent1);
     
-    Method method = scheduler.getClass().getDeclaredMethod("getNodes");
-    @SuppressWarnings("unchecked")
-    Map<NodeId, FiCaSchedulerNode> schedulerNodes = 
-        (Map<NodeId, FiCaSchedulerNode>) method.invoke(scheduler);
-    assertEquals(schedulerNodes.values().size(), 1);
+    assertEquals(scheduler.getNumClusterNodes(), 1);
     
     Resource newResource = Resources.createResource(1024, 4);
     
@@ -345,9 +336,9 @@ public class TestFifoScheduler {
     scheduler.handle(node0ResourceUpdate);
     
     // SchedulerNode's total resource and available resource are changed.
-    assertEquals(schedulerNodes.get(node0.getNodeID()).getTotalResource()
-        .getMemory(), 1024);
-    assertEquals(schedulerNodes.get(node0.getNodeID()).
+    assertEquals(1024, scheduler.getNodeTracker().getNode(node0.getNodeID())
+        .getTotalResource().getMemory());
+    assertEquals(1024, scheduler.getNodeTracker().getNode(node0.getNodeID()).
         getUnallocatedResource().getMemory(), 1024);
     QueueInfo queueInfo = scheduler.getQueueInfo(null, false, false);
     Assert.assertEquals(0.0f, queueInfo.getCurrentCapacity(), 0.0f);


[36/46] hadoop git commit: YARN-4108. CapacityScheduler: Improve preemption to only kill containers that would satisfy the incoming request. (Wangda Tan)

Posted by ae...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/ae14e5d0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/preemption/PreemptableQueue.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/preemption/PreemptableQueue.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/preemption/PreemptableQueue.java
new file mode 100644
index 0000000..19148d7
--- /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/preemption/PreemptableQueue.java
@@ -0,0 +1,102 @@
+/**
+ * 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
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
+ * 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.preemption;
+
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
+import org.apache.hadoop.yarn.util.resource.Resources;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.ConcurrentSkipListMap;
+
+public class PreemptableQueue {
+  // Partition -> killable resources and containers
+  private Map<String, Resource> totalKillableResources = new HashMap<>();
+  private Map<String, Map<ContainerId, RMContainer>> killableContainers =
+      new HashMap<>();
+  private PreemptableQueue parent;
+
+  public PreemptableQueue(PreemptableQueue parent) {
+    this.parent = parent;
+  }
+
+  public PreemptableQueue(Map<String, Resource> totalKillableResources,
+      Map<String, Map<ContainerId, RMContainer>> killableContainers) {
+    this.totalKillableResources = totalKillableResources;
+    this.killableContainers = killableContainers;
+  }
+
+  void addKillableContainer(KillableContainer container) {
+    String partition = container.getNodePartition();
+    if (!totalKillableResources.containsKey(partition)) {
+      totalKillableResources.put(partition, Resources.createResource(0));
+      killableContainers.put(partition,
+          new ConcurrentSkipListMap<ContainerId, RMContainer>());
+    }
+
+    RMContainer c = container.getRMContainer();
+    Resources.addTo(totalKillableResources.get(partition),
+        c.getAllocatedResource());
+    killableContainers.get(partition).put(c.getContainerId(), c);
+
+    if (null != parent) {
+      parent.addKillableContainer(container);
+    }
+  }
+
+  void removeKillableContainer(KillableContainer container) {
+    String partition = container.getNodePartition();
+    Map<ContainerId, RMContainer> partitionKillableContainers =
+        killableContainers.get(partition);
+    if (partitionKillableContainers != null) {
+      RMContainer rmContainer = partitionKillableContainers.remove(
+          container.getRMContainer().getContainerId());
+      if (null != rmContainer) {
+        Resources.subtractFrom(totalKillableResources.get(partition),
+            rmContainer.getAllocatedResource());
+      }
+    }
+
+    if (null != parent) {
+      parent.removeKillableContainer(container);
+    }
+  }
+
+  public Resource getKillableResource(String partition) {
+    Resource res = totalKillableResources.get(partition);
+    return res == null ? Resources.none() : res;
+  }
+
+  @SuppressWarnings("unchecked")
+  public Map<ContainerId, RMContainer> getKillableContainers(String partition) {
+    Map<ContainerId, RMContainer> map = killableContainers.get(partition);
+    return map == null ? Collections.EMPTY_MAP : map;
+  }
+
+  public Map<String, Map<ContainerId, RMContainer>> getKillableContainers() {
+    return killableContainers;
+  }
+
+  Map<String, Resource> getTotalKillableResources() {
+    return totalKillableResources;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ae14e5d0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/preemption/PreemptionManager.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/preemption/PreemptionManager.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/preemption/PreemptionManager.java
new file mode 100644
index 0000000..a9f02a5
--- /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/preemption/PreemptionManager.java
@@ -0,0 +1,165 @@
+/**
+ * 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
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
+ * 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.preemption;
+
+import com.google.common.annotations.VisibleForTesting;
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CSQueue;
+import org.apache.hadoop.yarn.util.resource.Resources;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+
+public class PreemptionManager {
+  private ReentrantReadWriteLock.ReadLock readLock;
+  private ReentrantReadWriteLock.WriteLock writeLock;
+  private Map<String, PreemptableQueue> entities = new HashMap<>();
+
+  public PreemptionManager() {
+    ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
+    readLock = lock.readLock();
+    writeLock = lock.writeLock();
+  }
+
+  public void refreshQueues(CSQueue parent, CSQueue current) {
+    try {
+      writeLock.lock();
+      PreemptableQueue parentEntity = null;
+      if (parent != null) {
+        parentEntity = entities.get(parent.getQueueName());
+      }
+
+      if (!entities.containsKey(current.getQueueName())) {
+        entities.put(current.getQueueName(),
+            new PreemptableQueue(parentEntity));
+      }
+
+      if (current.getChildQueues() != null) {
+        for (CSQueue child : current.getChildQueues()) {
+          refreshQueues(current, child);
+        }
+      }
+    }
+    finally {
+      writeLock.unlock();
+    }
+  }
+
+  public void addKillableContainer(KillableContainer container) {
+    try {
+      writeLock.lock();
+      PreemptableQueue entity = entities.get(container.getLeafQueueName());
+      if (null != entity) {
+        entity.addKillableContainer(container);
+      }
+    }
+    finally {
+      writeLock.unlock();
+    }
+  }
+
+  public void removeKillableContainer(KillableContainer container) {
+    try {
+      writeLock.lock();
+      PreemptableQueue entity = entities.get(container.getLeafQueueName());
+      if (null != entity) {
+        entity.removeKillableContainer(container);
+      }
+    }
+    finally {
+      writeLock.unlock();
+    }
+  }
+
+  public void moveKillableContainer(KillableContainer oldContainer,
+      KillableContainer newContainer) {
+    // TODO, will be called when partition of the node changed OR
+    // container moved to different queue
+  }
+
+  public void updateKillableContainerResource(KillableContainer container,
+      Resource oldResource, Resource newResource) {
+    // TODO, will be called when container's resource changed
+  }
+
+  @VisibleForTesting
+  public Map<ContainerId, RMContainer> getKillableContainersMap(
+      String queueName, String partition) {
+    try {
+      readLock.lock();
+      PreemptableQueue entity = entities.get(queueName);
+      if (entity != null) {
+        Map<ContainerId, RMContainer> containers =
+            entity.getKillableContainers().get(partition);
+        if (containers != null) {
+          return containers;
+        }
+      }
+      return Collections.emptyMap();
+    }
+    finally {
+      readLock.unlock();
+    }
+  }
+
+  public Iterator<RMContainer> getKillableContainers(String queueName,
+      String partition) {
+    return getKillableContainersMap(queueName, partition).values().iterator();
+  }
+
+  public Resource getKillableResource(String queueName, String partition) {
+    try {
+      readLock.lock();
+      PreemptableQueue entity = entities.get(queueName);
+      if (entity != null) {
+        Resource res = entity.getTotalKillableResources().get(partition);
+        if (res == null || res.equals(Resources.none())) {
+          return Resources.none();
+        }
+        return Resources.clone(res);
+      }
+      return Resources.none();
+    }
+    finally {
+      readLock.unlock();
+    }
+  }
+
+  public Map<String, PreemptableQueue> getShallowCopyOfPreemptableEntities() {
+    try {
+      readLock.lock();
+      Map<String, PreemptableQueue> map = new HashMap<>();
+      for (Map.Entry<String, PreemptableQueue> entry : entities.entrySet()) {
+        String key = entry.getKey();
+        PreemptableQueue entity = entry.getValue();
+        map.put(key, new PreemptableQueue(
+            new HashMap<>(entity.getTotalKillableResources()),
+            new HashMap<>(entity.getKillableContainers())));
+      }
+      return map;
+    } finally {
+      readLock.unlock();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ae14e5d0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/common/AssignmentInformation.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/AssignmentInformation.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/AssignmentInformation.java
index 5158255..aad3bc7 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/AssignmentInformation.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/AssignmentInformation.java
@@ -120,9 +120,9 @@ public class AssignmentInformation {
   }
 
   private ContainerId getFirstContainerIdFromOperation(Operation op) {
-    if (null != operationDetails.get(Operation.ALLOCATION)) {
+    if (null != operationDetails.get(op)) {
       List<AssignmentDetails> assignDetails =
-          operationDetails.get(Operation.ALLOCATION);
+          operationDetails.get(op);
       if (!assignDetails.isEmpty()) {
         return assignDetails.get(0).containerId;
       }
@@ -131,7 +131,7 @@ public class AssignmentInformation {
   }
 
   public ContainerId getFirstAllocatedOrReservedContainerId() {
-    ContainerId containerId = null;
+    ContainerId containerId;
     containerId = getFirstContainerIdFromOperation(Operation.ALLOCATION);
     if (null != containerId) {
       return containerId;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ae14e5d0/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 4d563cd..f474aad 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
@@ -59,6 +59,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CSAMCont
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CSAssignment;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityHeadroomProvider;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerContext;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.LeafQueue;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueueCapacities;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.SchedulingMode;
@@ -94,6 +95,7 @@ public class FiCaSchedulerApp extends SchedulerApplicationAttempt {
    * to hold the message if its app doesn't not get container from a node
    */
   private String appSkipNodeDiagnostics;
+  private CapacitySchedulerContext capacitySchedulerContext;
 
   public FiCaSchedulerApp(ApplicationAttemptId applicationAttemptId, 
       String user, Queue queue, ActiveUsersManager activeUsersManager,
@@ -138,28 +140,30 @@ public class FiCaSchedulerApp extends SchedulerApplicationAttempt {
     }
     
     containerAllocator = new ContainerAllocator(this, rc, rmContext);
+
+    if (scheduler instanceof CapacityScheduler) {
+      capacitySchedulerContext = (CapacitySchedulerContext) scheduler;
+    }
   }
 
-  synchronized public boolean containerCompleted(RMContainer rmContainer,
+  public synchronized boolean containerCompleted(RMContainer rmContainer,
       ContainerStatus containerStatus, RMContainerEventType event,
       String partition) {
+    ContainerId containerId = rmContainer.getContainerId();
 
     // Remove from the list of containers
-    if (null == liveContainers.remove(rmContainer.getContainerId())) {
+    if (null == liveContainers.remove(containerId)) {
       return false;
     }
-    
+
     // Remove from the list of newly allocated containers if found
     newlyAllocatedContainers.remove(rmContainer);
 
-    Container container = rmContainer.getContainer();
-    ContainerId containerId = container.getId();
-
     // Inform the container
     rmContainer.handle(
         new RMContainerFinishedEvent(containerId, containerStatus, event));
 
-    containersToPreempt.remove(rmContainer.getContainerId());
+    containersToPreempt.remove(containerId);
 
     RMAuditLogger.logSuccess(getUser(),
         AuditConstants.RELEASE_CONTAINER, "SchedulerApp",
@@ -176,7 +180,7 @@ public class FiCaSchedulerApp extends SchedulerApplicationAttempt {
     return true;
   }
 
-  synchronized public RMContainer allocate(NodeType type, FiCaSchedulerNode node,
+  public synchronized RMContainer allocate(NodeType type, FiCaSchedulerNode node,
       Priority priority, ResourceRequest request, 
       Container container) {
 
@@ -200,7 +204,9 @@ public class FiCaSchedulerApp extends SchedulerApplicationAttempt {
 
     // Add it to allContainers list.
     newlyAllocatedContainers.add(rmContainer);
-    liveContainers.put(container.getId(), rmContainer);    
+
+    ContainerId containerId = container.getId();
+    liveContainers.put(containerId, rmContainer);
 
     // Update consumption and track allocations
     List<ResourceRequest> resourceRequestList = appSchedulingInfo.allocate(
@@ -213,17 +219,17 @@ public class FiCaSchedulerApp extends SchedulerApplicationAttempt {
 
     // Inform the container
     rmContainer.handle(
-        new RMContainerEvent(container.getId(), RMContainerEventType.START));
+        new RMContainerEvent(containerId, RMContainerEventType.START));
 
     if (LOG.isDebugEnabled()) {
       LOG.debug("allocate: applicationAttemptId=" 
-          + container.getId().getApplicationAttemptId() 
-          + " container=" + container.getId() + " host="
+          + containerId.getApplicationAttemptId()
+          + " container=" + containerId + " host="
           + container.getNodeId().getHost() + " type=" + type);
     }
     RMAuditLogger.logSuccess(getUser(),
         AuditConstants.ALLOC_CONTAINER, "SchedulerApp",
-        getApplicationId(), container.getId());
+        getApplicationId(), containerId);
     
     return rmContainer;
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ae14e5d0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/common/fica/FiCaSchedulerNode.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/FiCaSchedulerNode.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/FiCaSchedulerNode.java
index fe6db47..1d0e78a 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/FiCaSchedulerNode.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/FiCaSchedulerNode.java
@@ -18,22 +18,29 @@
 
 package org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica;
 
-
-import java.util.Set;
-
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
+import org.apache.hadoop.yarn.api.records.Container;
+import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.Priority;
+import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.nodelabels.CommonNodeLabelsManager;
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplicationAttempt;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNode;
+import org.apache.hadoop.yarn.util.resource.Resources;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Set;
 
 public class FiCaSchedulerNode extends SchedulerNode {
 
   private static final Log LOG = LogFactory.getLog(FiCaSchedulerNode.class);
+  private Map<ContainerId, RMContainer> killableContainers = new HashMap<>();
+  private Resource totalKillableResources = Resource.newInstance(0, 0);
   
   public FiCaSchedulerNode(RMNode node, boolean usePortForNodeName,
       Set<String> nodeLabels) {
@@ -92,7 +99,6 @@ public class FiCaSchedulerNode extends SchedulerNode {
   @Override
   public synchronized void unreserveResource(
       SchedulerApplicationAttempt application) {
-
     // adding NP checks as this can now be called for preemption
     if (getReservedContainer() != null
         && getReservedContainer().getContainer() != null
@@ -115,4 +121,55 @@ public class FiCaSchedulerNode extends SchedulerNode {
     }
     setReservedContainer(null);
   }
+
+  // According to decisions from preemption policy, mark the container to killable
+  public synchronized void markContainerToKillable(ContainerId containerId) {
+    RMContainer c = launchedContainers.get(containerId);
+    if (c != null && !killableContainers.containsKey(containerId)) {
+      killableContainers.put(containerId, c);
+      Resources.addTo(totalKillableResources, c.getAllocatedResource());
+    }
+  }
+
+  // According to decisions from preemption policy, mark the container to
+  // non-killable
+  public synchronized void markContainerToNonKillable(ContainerId containerId) {
+    RMContainer c = launchedContainers.get(containerId);
+    if (c != null && killableContainers.containsKey(containerId)) {
+      killableContainers.remove(containerId);
+      Resources.subtractFrom(totalKillableResources, c.getAllocatedResource());
+    }
+  }
+
+  @Override
+  protected synchronized void updateResource(
+      Container container) {
+    super.updateResource(container);
+    if (killableContainers.containsKey(container.getId())) {
+      Resources.subtractFrom(totalKillableResources, container.getResource());
+      killableContainers.remove(container.getId());
+    }
+  }
+
+  @Override
+  protected synchronized void changeContainerResource(ContainerId containerId,
+      Resource deltaResource, boolean increase) {
+    super.changeContainerResource(containerId, deltaResource, increase);
+
+    if (killableContainers.containsKey(containerId)) {
+      if (increase) {
+        Resources.addTo(totalKillableResources, deltaResource);
+      } else {
+        Resources.subtractFrom(totalKillableResources, deltaResource);
+      }
+    }
+  }
+
+  public synchronized Resource getTotalKillableResources() {
+    return totalKillableResources;
+  }
+
+  public synchronized Map<ContainerId, RMContainer> getKillableContainers() {
+    return killableContainers;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ae14e5d0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/event/SchedulerEventType.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/event/SchedulerEventType.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/event/SchedulerEventType.java
index 9cf09e9..35b7c14 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/event/SchedulerEventType.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/event/SchedulerEventType.java
@@ -38,10 +38,15 @@ public enum SchedulerEventType {
   // Source: ContainerAllocationExpirer
   CONTAINER_EXPIRED,
 
-  // Source: SchedulingEditPolicy
+  /* Source: SchedulingEditPolicy */
   KILL_RESERVED_CONTAINER,
-  MARK_CONTAINER_FOR_PREEMPTION, // Mark a container for preemption
-                                 // in the near future
-  KILL_PREEMPTED_CONTAINER // Kill a container previously marked for
-                           // preemption
+
+  // Mark a container for preemption
+  MARK_CONTAINER_FOR_PREEMPTION,
+
+  // Mark a for-preemption container killable
+  MARK_CONTAINER_FOR_KILLABLE,
+
+  // Cancel a killable container
+  MARK_CONTAINER_FOR_NONKILLABLE
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ae14e5d0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMDispatcher.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/TestRMDispatcher.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMDispatcher.java
index d9306dd..c944752 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMDispatcher.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMDispatcher.java
@@ -59,7 +59,7 @@ public class TestRMDispatcher {
       rmDispatcher.getEventHandler().handle(event1);
       ContainerPreemptEvent event2 =
           new ContainerPreemptEvent(appAttemptId, container,
-            SchedulerEventType.KILL_PREEMPTED_CONTAINER);
+            SchedulerEventType.MARK_CONTAINER_FOR_KILLABLE);
       rmDispatcher.getEventHandler().handle(event2);
       ContainerPreemptEvent event3 =
           new ContainerPreemptEvent(appAttemptId, container,
@@ -70,7 +70,7 @@ public class TestRMDispatcher {
       verify(sched, times(3)).handle(any(SchedulerEvent.class));
       verify(sched).killReservedContainer(container);
       verify(sched).markContainerForPreemption(appAttemptId, container);
-      verify(sched).killPreemptedContainer(container);
+      verify(sched).markContainerForKillable(container);
     } catch (InterruptedException e) {
       Assert.fail();
     } finally {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ae14e5d0/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 028afb1..3057615 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
@@ -2352,7 +2352,7 @@ public class TestRMRestart extends ParameterizedSchedulerTestBase {
       FiCaSchedulerApp schedulerAppAttempt = cs.getSchedulerApplications()
           .get(app0.getApplicationId()).getCurrentAppAttempt();
       // kill app0-attempt
-      cs.killPreemptedContainer(schedulerAppAttempt.getRMContainer(
+      cs.markContainerForKillable(schedulerAppAttempt.getRMContainer(
           app0.getCurrentAppAttempt().getMasterContainer().getId()));
       am0.waitForState(RMAppAttemptState.FAILED);
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ae14e5d0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/applicationsmanager/TestAMRestart.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/TestAMRestart.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/applicationsmanager/TestAMRestart.java
index 5035afe..16f3f60 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/applicationsmanager/TestAMRestart.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/applicationsmanager/TestAMRestart.java
@@ -63,7 +63,6 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEv
 import org.apache.hadoop.yarn.server.utils.BuilderUtils;
 import org.apache.hadoop.yarn.util.ControlledClock;
 import org.apache.hadoop.yarn.util.Records;
-import org.apache.hadoop.yarn.util.SystemClock;
 import org.apache.hadoop.yarn.util.resource.Resources;
 import org.junit.Assert;
 import org.junit.Test;
@@ -566,7 +565,7 @@ public class TestAMRestart {
     ContainerId amContainer =
         ContainerId.newContainerId(am1.getApplicationAttemptId(), 1);
     // Preempt the first attempt;
-    scheduler.killPreemptedContainer(scheduler.getRMContainer(amContainer));
+    scheduler.markContainerForKillable(scheduler.getRMContainer(amContainer));
 
     am1.waitForState(RMAppAttemptState.FAILED);
     Assert.assertTrue(! attempt1.shouldCountTowardsMaxAttemptRetry());
@@ -582,7 +581,7 @@ public class TestAMRestart {
     // Preempt the second attempt.
     ContainerId amContainer2 =
         ContainerId.newContainerId(am2.getApplicationAttemptId(), 1);
-    scheduler.killPreemptedContainer(scheduler.getRMContainer(amContainer2));
+    scheduler.markContainerForKillable(scheduler.getRMContainer(amContainer2));
 
     am2.waitForState(RMAppAttemptState.FAILED);
     Assert.assertTrue(! attempt2.shouldCountTowardsMaxAttemptRetry());
@@ -677,7 +676,7 @@ public class TestAMRestart {
         ContainerId.newContainerId(am1.getApplicationAttemptId(), 1);
 
     // Forcibly preempt the am container;
-    scheduler.killPreemptedContainer(scheduler.getRMContainer(amContainer));
+    scheduler.markContainerForKillable(scheduler.getRMContainer(amContainer));
 
     am1.waitForState(RMAppAttemptState.FAILED);
     Assert.assertTrue(! attempt1.shouldCountTowardsMaxAttemptRetry());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ae14e5d0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/TestProportionalCapacityPreemptionPolicy.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/monitor/capacity/TestProportionalCapacityPreemptionPolicy.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/TestProportionalCapacityPreemptionPolicy.java
index 13f267d..e9129de 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/TestProportionalCapacityPreemptionPolicy.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/TestProportionalCapacityPreemptionPolicy.java
@@ -23,7 +23,7 @@ import static org.apache.hadoop.yarn.server.resourcemanager.monitor.capacity.Pro
 import static org.apache.hadoop.yarn.server.resourcemanager.monitor.capacity.ProportionalCapacityPreemptionPolicy.OBSERVE_ONLY;
 import static org.apache.hadoop.yarn.server.resourcemanager.monitor.capacity.ProportionalCapacityPreemptionPolicy.TOTAL_PREEMPTION_PER_ROUND;
 import static org.apache.hadoop.yarn.server.resourcemanager.monitor.capacity.ProportionalCapacityPreemptionPolicy.WAIT_TIME_BEFORE_KILL;
-import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEventType.KILL_PREEMPTED_CONTAINER;
+import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEventType.MARK_CONTAINER_FOR_KILLABLE;
 import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEventType.MARK_CONTAINER_FOR_PREEMPTION;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNotNull;
@@ -75,6 +75,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.Capacity
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.LeafQueue;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.ParentQueue;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueueCapacities;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.preemption.PreemptionManager;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerApp;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.ContainerPreemptEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEvent;
@@ -167,6 +168,7 @@ public class TestProportionalCapacityPreemptionPolicy {
     when(mCS.getConfiguration()).thenReturn(schedConf);
     rmContext = mock(RMContext.class);
     when(mCS.getRMContext()).thenReturn(rmContext);
+    when(mCS.getPreemptionManager()).thenReturn(new PreemptionManager());
     when(rmContext.getNodeLabelManager()).thenReturn(lm);
     mDisp = mock(EventHandler.class);
     Dispatcher disp = mock(Dispatcher.class);
@@ -289,7 +291,7 @@ public class TestProportionalCapacityPreemptionPolicy {
     List<ContainerPreemptEvent> events = evtCaptor.getAllValues();
     for (ContainerPreemptEvent e : events.subList(20, 20)) {
       assertEquals(appC, e.getAppId());
-      assertEquals(KILL_PREEMPTED_CONTAINER, e.getType());
+      assertEquals(MARK_CONTAINER_FOR_KILLABLE, e.getType());
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ae14e5d0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/TestProportionalCapacityPreemptionPolicyForNodePartitions.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/monitor/capacity/TestProportionalCapacityPreemptionPolicyForNodePartitions.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/TestProportionalCapacityPreemptionPolicyForNodePartitions.java
index 512f37c..21ea495 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/TestProportionalCapacityPreemptionPolicyForNodePartitions.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/TestProportionalCapacityPreemptionPolicyForNodePartitions.java
@@ -67,6 +67,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.Capacity
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.LeafQueue;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.ParentQueue;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueueCapacities;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.preemption.PreemptionManager;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerApp;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.policy.OrderingPolicy;
@@ -123,6 +124,7 @@ public class TestProportionalCapacityPreemptionPolicyForNodePartitions {
     mClock = mock(Clock.class);
     cs = mock(CapacityScheduler.class);
     when(cs.getResourceCalculator()).thenReturn(rc);
+    when(cs.getPreemptionManager()).thenReturn(new PreemptionManager());
 
     nlm = mock(RMNodeLabelsManager.class);
     mDisp = mock(EventHandler.class);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ae14e5d0/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 0b32676..171196f 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
@@ -52,6 +52,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttempt;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ActiveUsersManager;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceLimits;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.preemption.PreemptionManager;
 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.server.resourcemanager.security.RMContainerTokenSecretManager;
@@ -264,6 +265,7 @@ public class TestApplicationLimits {
         thenReturn(CapacityScheduler.nonPartitionedQueueComparator);
     when(csContext.getResourceCalculator()).thenReturn(resourceCalculator);
     when(csContext.getRMContext()).thenReturn(rmContext);
+    when(csContext.getPreemptionManager()).thenReturn(new PreemptionManager());
     
     // Say cluster has 100 nodes of 16G each
     Resource clusterResource = 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ae14e5d0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestApplicationPriority.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/TestApplicationPriority.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/TestApplicationPriority.java
index 1569a12..d8161f8 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/TestApplicationPriority.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/TestApplicationPriority.java
@@ -205,7 +205,7 @@ public class TestApplicationPriority {
       if (++counter > 2) {
         break;
       }
-      cs.killPreemptedContainer(schedulerAppAttempt.getRMContainer(c.getId()));
+      cs.markContainerForKillable(schedulerAppAttempt.getRMContainer(c.getId()));
     }
 
     // check node report, 12 GB used and 4 GB available
@@ -512,7 +512,7 @@ public class TestApplicationPriority {
       if (++counter > 2) {
         break;
       }
-      cs.killPreemptedContainer(schedulerAppAttemptApp1.getRMContainer(c.getId()));
+      cs.markContainerForKillable(schedulerAppAttemptApp1.getRMContainer(c.getId()));
       iterator.remove();
     }
 
@@ -542,7 +542,7 @@ public class TestApplicationPriority {
       if (++counter > 1) {
         break;
       }
-      cs.killPreemptedContainer(schedulerAppAttemptApp1.getRMContainer(c.getId()));
+      cs.markContainerForKillable(schedulerAppAttemptApp1.getRMContainer(c.getId()));
       iterator.remove();
     }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ae14e5d0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacityScheduler.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/TestCapacityScheduler.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/TestCapacityScheduler.java
index b6c005b..16ba607 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/TestCapacityScheduler.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/TestCapacityScheduler.java
@@ -1188,7 +1188,7 @@ public class TestCapacityScheduler {
 
     // kill the 3 containers
     for (Container c : allocatedContainers) {
-      cs.killPreemptedContainer(schedulerAppAttempt.getRMContainer(c.getId()));
+      cs.markContainerForKillable(schedulerAppAttempt.getRMContainer(c.getId()));
     }
 
     // check values
@@ -1197,7 +1197,7 @@ public class TestCapacityScheduler {
         Resource.newInstance(CONTAINER_MEMORY * 3, 3), false, 3);
 
     // kill app0-attempt0 AM container
-    cs.killPreemptedContainer(schedulerAppAttempt.getRMContainer(app0
+    cs.markContainerForKillable(schedulerAppAttempt.getRMContainer(app0
         .getCurrentAppAttempt().getMasterContainer().getId()));
 
     // wait for app0 failed
@@ -1220,7 +1220,7 @@ public class TestCapacityScheduler {
     allocatedContainers =
         am1.allocateAndWaitForContainers(3, CONTAINER_MEMORY, nm1);
     for (Container c : allocatedContainers) {
-      cs.killPreemptedContainer(schedulerAppAttempt.getRMContainer(c.getId()));
+      cs.markContainerForKillable(schedulerAppAttempt.getRMContainer(c.getId()));
     }
 
     // check values
@@ -1269,7 +1269,7 @@ public class TestCapacityScheduler {
     }
 
     // Call killContainer to preempt the container
-    cs.killPreemptedContainer(rmContainer);
+    cs.markContainerForKillable(rmContainer);
 
     Assert.assertEquals(3, requests.size());
     for (ResourceRequest request : requests) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ae14e5d0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacitySchedulerPreemption.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/TestCapacitySchedulerPreemption.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/TestCapacitySchedulerPreemption.java
new file mode 100644
index 0000000..bea7797
--- /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/TestCapacitySchedulerPreemption.java
@@ -0,0 +1,677 @@
+/**
+ * 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 com.google.common.collect.Sets;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.service.Service;
+import org.apache.hadoop.yarn.api.records.ContainerId;
+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.RMActiveServices;
+import org.apache.hadoop.yarn.server.resourcemanager.monitor.SchedulingEditPolicy;
+import org.apache.hadoop.yarn.server.resourcemanager.monitor.SchedulingMonitor;
+import org.apache.hadoop.yarn.server.resourcemanager.monitor.capacity.ProportionalCapacityPreemptionPolicy;
+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.rmnode.RMNode;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.preemption.PreemptionManager;
+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.util.Clock;
+import org.apache.hadoop.yarn.util.resource.Resources;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+public class TestCapacitySchedulerPreemption {
+  private static final Log LOG = LogFactory.getLog(
+      TestCapacitySchedulerPreemption.class);
+
+  private final int GB = 1024;
+
+  private Configuration conf;
+
+  RMNodeLabelsManager mgr;
+
+  Clock clock;
+
+  @Before
+  public void setUp() throws Exception {
+    conf = new YarnConfiguration();
+    conf.setClass(YarnConfiguration.RM_SCHEDULER, CapacityScheduler.class,
+        ResourceScheduler.class);
+    conf.setBoolean(YarnConfiguration.RM_SCHEDULER_ENABLE_MONITORS, true);
+    conf.setClass(YarnConfiguration.RM_SCHEDULER_MONITOR_POLICIES,
+        ProportionalCapacityPreemptionPolicy.class, SchedulingEditPolicy.class);
+    conf = TestUtils.getConfigurationWithMultipleQueues(this.conf);
+
+    // Set preemption related configurations
+    conf.setInt(ProportionalCapacityPreemptionPolicy.WAIT_TIME_BEFORE_KILL,
+        0);
+    conf.setBoolean(CapacitySchedulerConfiguration.LAZY_PREEMPTION_ENALBED,
+        true);
+    conf.setFloat(
+        ProportionalCapacityPreemptionPolicy.TOTAL_PREEMPTION_PER_ROUND, 1.0f);
+    conf.setFloat(
+        ProportionalCapacityPreemptionPolicy.NATURAL_TERMINATION_FACTOR, 1.0f);
+    mgr = new NullRMNodeLabelsManager();
+    mgr.init(this.conf);
+    clock = mock(Clock.class);
+    when(clock.getTime()).thenReturn(0L);
+  }
+
+  private SchedulingEditPolicy getSchedulingEditPolicy(MockRM rm) {
+    RMActiveServices activeServices = rm.getRMActiveService();
+    SchedulingMonitor mon = null;
+    for (Service service : activeServices.getServices()) {
+      if (service instanceof SchedulingMonitor) {
+        mon = (SchedulingMonitor) service;
+        break;
+      }
+    }
+
+    if (mon != null) {
+      return mon.getSchedulingEditPolicy();
+    }
+    return null;
+  }
+
+  @Test (timeout = 60000)
+  public void testSimplePreemption() throws Exception {
+    /**
+     * Test case: Submit two application (app1/app2) to different queues, queue
+     * structure:
+     *
+     * <pre>
+     *             Root
+     *            /  |  \
+     *           a   b   c
+     *          10   20  70
+     * </pre>
+     *
+     * 1) Two nodes in the cluster, each of them has 4G.
+     *
+     * 2) app1 submit to queue-a first, it asked 7 * 1G containers, so there's no
+     * more resource available.
+     *
+     * 3) app2 submit to queue-c, ask for one 1G container (for AM)
+     *
+     * Now the cluster is fulfilled.
+     *
+     * 4) app2 asks for another 1G container, system will preempt one container
+     * from app1, and app2 will receive the preempted container
+     */
+    MockRM rm1 = new MockRM(conf);
+    rm1.getRMContext().setNodeLabelManager(mgr);
+    rm1.start();
+
+    MockNM nm1 = rm1.registerNode("h1:1234", 4 * GB);
+    MockNM nm2 = rm1.registerNode("h2:1234", 4 * GB);
+    CapacityScheduler cs = (CapacityScheduler) rm1.getResourceScheduler();
+    RMNode rmNode1 = rm1.getRMContext().getRMNodes().get(nm1.getNodeId());
+    RMNode rmNode2 = rm1.getRMContext().getRMNodes().get(nm2.getNodeId());
+
+    // launch an app to queue, AM container should be launched in nm1
+    RMApp app1 = rm1.submitApp(1 * GB, "app", "user", null, "a");
+    MockAM am1 = MockRM.launchAndRegisterAM(app1, rm1, nm1);
+
+    am1.allocate("*", 1 * GB, 7, new ArrayList<ContainerId>());
+
+    // Do allocation 3 times for node1/node2
+    for (int i = 0; i < 3; i++) {
+      cs.handle(new NodeUpdateSchedulerEvent(rmNode1));
+      cs.handle(new NodeUpdateSchedulerEvent(rmNode2));
+    }
+
+    // App1 should have 7 containers now, and no available resource for cluster
+    FiCaSchedulerApp schedulerApp1 = cs.getApplicationAttempt(
+        am1.getApplicationAttemptId());
+    Assert.assertEquals(7, schedulerApp1.getLiveContainers().size());
+
+    // Submit app2 to queue-c and asks for a 1G container for AM
+    RMApp app2 = rm1.submitApp(1 * GB, "app", "user", null, "c");
+    MockAM am2 = MockRM.launchAndRegisterAM(app2, rm1, nm2);
+
+    // NM1/NM2 has available resource = 0G
+    Assert.assertEquals(0 * GB, cs.getNode(nm1.getNodeId())
+        .getUnallocatedResource().getMemory());
+    Assert.assertEquals(0 * GB, cs.getNode(nm2.getNodeId())
+        .getUnallocatedResource().getMemory());
+
+    // AM asks for a 1 * GB container
+    am2.allocate(Arrays.asList(ResourceRequest
+        .newInstance(Priority.newInstance(1), ResourceRequest.ANY,
+            Resources.createResource(1 * GB), 1)), null);
+
+    // Get edit policy and do one update
+    SchedulingEditPolicy editPolicy = getSchedulingEditPolicy(rm1);
+
+    // Call edit schedule twice, and check if one container from app1 marked
+    // to be "killable"
+    editPolicy.editSchedule();
+    editPolicy.editSchedule();
+
+    PreemptionManager pm = cs.getPreemptionManager();
+    Map<ContainerId, RMContainer> killableContainers =
+        waitKillableContainersSize(pm, "a", RMNodeLabelsManager.NO_LABEL, 1);
+    Assert.assertEquals(1, killableContainers.size());
+    Assert.assertEquals(killableContainers.entrySet().iterator().next().getKey()
+        .getApplicationAttemptId(), am1.getApplicationAttemptId());
+
+    // Call CS.handle once to see if container preempted
+    cs.handle(new NodeUpdateSchedulerEvent(rmNode2));
+
+    FiCaSchedulerApp schedulerApp2 = cs.getApplicationAttempt(
+        am2.getApplicationAttemptId());
+
+    // App1 has 6 containers, and app2 has 2 containers
+    Assert.assertEquals(6, schedulerApp1.getLiveContainers().size());
+    Assert.assertEquals(2, schedulerApp2.getLiveContainers().size());
+
+    rm1.close();
+  }
+
+  @Test (timeout = 60000)
+  public void testPreemptionConsidersNodeLocalityDelay()
+      throws Exception {
+    /**
+     * Test case: same as testSimplePreemption steps 1-3.
+     *
+     * Step 4: app2 asks for 1G container with locality specified, so it needs
+     * to wait for missed-opportunity before get scheduled.
+     * Check if system waits missed-opportunity before finish killable container
+     */
+    MockRM rm1 = new MockRM(conf);
+    rm1.getRMContext().setNodeLabelManager(mgr);
+    rm1.start();
+    MockNM nm1 = rm1.registerNode("h1:1234", 4 * GB);
+    MockNM nm2 = rm1.registerNode("h2:1234", 4 * GB);
+    CapacityScheduler cs = (CapacityScheduler) rm1.getResourceScheduler();
+    RMNode rmNode1 = rm1.getRMContext().getRMNodes().get(nm1.getNodeId());
+    RMNode rmNode2 = rm1.getRMContext().getRMNodes().get(nm2.getNodeId());
+
+    // launch an app to queue, AM container should be launched in nm1
+    RMApp app1 = rm1.submitApp(1 * GB, "app", "user", null, "a");
+    MockAM am1 = MockRM.launchAndRegisterAM(app1, rm1, nm1);
+
+    am1.allocate("*", 1 * GB, 6, new ArrayList<ContainerId>());
+
+    // Do allocation 3 times for node1/node2
+    for (int i = 0; i < 3; i++) {
+      cs.handle(new NodeUpdateSchedulerEvent(rmNode1));
+      cs.handle(new NodeUpdateSchedulerEvent(rmNode2));
+    }
+
+    // App1 should have 7 containers now, and no available resource for cluster
+    FiCaSchedulerApp schedulerApp1 = cs.getApplicationAttempt(
+        am1.getApplicationAttemptId());
+    Assert.assertEquals(7, schedulerApp1.getLiveContainers().size());
+
+    // Submit app2 to queue-c and asks for a 1G container for AM
+    RMApp app2 = rm1.submitApp(1 * GB, "app", "user", null, "c");
+    MockAM am2 = MockRM.launchAndRegisterAM(app2, rm1, nm2);
+
+    // NM1/NM2 has available resource = 0G
+    Assert.assertEquals(0 * GB, cs.getNode(nm1.getNodeId())
+        .getUnallocatedResource().getMemory());
+    Assert.assertEquals(0 * GB, cs.getNode(nm2.getNodeId())
+        .getUnallocatedResource().getMemory());
+
+    // AM asks for a 1 * GB container with unknown host and unknown rack
+    am2.allocate(Arrays.asList(ResourceRequest
+        .newInstance(Priority.newInstance(1), ResourceRequest.ANY,
+            Resources.createResource(1 * GB), 1), ResourceRequest
+        .newInstance(Priority.newInstance(1), "unknownhost",
+            Resources.createResource(1 * GB), 1), ResourceRequest
+        .newInstance(Priority.newInstance(1), "/default-rack",
+            Resources.createResource(1 * GB), 1)), null);
+
+    // Get edit policy and do one update
+    SchedulingEditPolicy editPolicy = getSchedulingEditPolicy(rm1);
+
+    // Call edit schedule twice, and check if one container from app1 marked
+    // to be "killable"
+    editPolicy.editSchedule();
+    editPolicy.editSchedule();
+
+    PreemptionManager pm = cs.getPreemptionManager();
+    Map<ContainerId, RMContainer> killableContainers =
+        waitKillableContainersSize(pm, "a", RMNodeLabelsManager.NO_LABEL, 1);
+    Assert.assertEquals(killableContainers.entrySet().iterator().next().getKey()
+        .getApplicationAttemptId(), am1.getApplicationAttemptId());
+
+    // Call CS.handle once to see if container preempted
+    cs.handle(new NodeUpdateSchedulerEvent(rmNode2));
+
+    FiCaSchedulerApp schedulerApp2 = cs.getApplicationAttempt(
+        am2.getApplicationAttemptId());
+
+    // App1 has 7 containers, and app2 has 1 containers (no container preempted)
+    Assert.assertEquals(7, schedulerApp1.getLiveContainers().size());
+    Assert.assertEquals(1, schedulerApp2.getLiveContainers().size());
+
+    // Do allocation again, one container will be preempted
+    cs.handle(new NodeUpdateSchedulerEvent(rmNode2));
+
+    // App1 has 6 containers, and app2 has 2 containers (new container allocated)
+    Assert.assertEquals(6, schedulerApp1.getLiveContainers().size());
+    Assert.assertEquals(2, schedulerApp2.getLiveContainers().size());
+
+    rm1.close();
+  }
+
+  @Test (timeout = 60000)
+  public void testPreemptionConsidersHardNodeLocality()
+      throws Exception {
+    /**
+     * Test case: same as testSimplePreemption steps 1-3.
+     *
+     * Step 4: app2 asks for 1G container with hard locality specified, and
+     *         asked host is not existed
+     * Confirm system doesn't preempt any container.
+     */
+    MockRM rm1 = new MockRM(conf);
+    rm1.getRMContext().setNodeLabelManager(mgr);
+    rm1.start();
+    MockNM nm1 = rm1.registerNode("h1:1234", 4 * GB);
+    MockNM nm2 = rm1.registerNode("h2:1234", 4 * GB);
+    CapacityScheduler cs = (CapacityScheduler) rm1.getResourceScheduler();
+    RMNode rmNode1 = rm1.getRMContext().getRMNodes().get(nm1.getNodeId());
+    RMNode rmNode2 = rm1.getRMContext().getRMNodes().get(nm2.getNodeId());
+
+    // launch an app to queue, AM container should be launched in nm1
+    RMApp app1 = rm1.submitApp(1 * GB, "app", "user", null, "a");
+    MockAM am1 = MockRM.launchAndRegisterAM(app1, rm1, nm1);
+
+    am1.allocate("*", 1 * GB, 6, new ArrayList<ContainerId>());
+
+    // Do allocation 3 times for node1/node2
+    for (int i = 0; i < 3; i++) {
+      cs.handle(new NodeUpdateSchedulerEvent(rmNode1));
+    }
+    for (int i = 0; i < 3; i++) {
+      cs.handle(new NodeUpdateSchedulerEvent(rmNode2));
+    }
+
+    // App1 should have 7 containers now, and no available resource for cluster
+    FiCaSchedulerApp schedulerApp1 = cs.getApplicationAttempt(
+        am1.getApplicationAttemptId());
+    Assert.assertEquals(7, schedulerApp1.getLiveContainers().size());
+
+    // Submit app2 to queue-c and asks for a 1G container for AM
+    RMApp app2 = rm1.submitApp(1 * GB, "app", "user", null, "c");
+    MockAM am2 = MockRM.launchAndRegisterAM(app2, rm1, nm2);
+
+    // NM1/NM2 has available resource = 0G
+    Assert.assertEquals(0 * GB, cs.getNode(nm1.getNodeId())
+        .getUnallocatedResource().getMemory());
+    Assert.assertEquals(0 * GB, cs.getNode(nm2.getNodeId())
+        .getUnallocatedResource().getMemory());
+
+    // AM asks for a 1 * GB container for h3 with hard locality,
+    // h3 doesn't exist in the cluster
+    am2.allocate(Arrays.asList(ResourceRequest
+        .newInstance(Priority.newInstance(1), ResourceRequest.ANY,
+            Resources.createResource(1 * GB), 1, true), ResourceRequest
+        .newInstance(Priority.newInstance(1), "h3",
+            Resources.createResource(1 * GB), 1, false), ResourceRequest
+        .newInstance(Priority.newInstance(1), "/default-rack",
+            Resources.createResource(1 * GB), 1, false)), null);
+
+    // Get edit policy and do one update
+    SchedulingEditPolicy editPolicy = getSchedulingEditPolicy(rm1);
+
+    // Call edit schedule twice, and check if one container from app1 marked
+    // to be "killable"
+    editPolicy.editSchedule();
+    editPolicy.editSchedule();
+
+    PreemptionManager pm = cs.getPreemptionManager();
+    Map<ContainerId, RMContainer> killableContainers =
+        waitKillableContainersSize(pm, "a", RMNodeLabelsManager.NO_LABEL, 1);
+    Assert.assertEquals(killableContainers.entrySet().iterator().next().getKey()
+        .getApplicationAttemptId(), am1.getApplicationAttemptId());
+
+    // Call CS.handle once to see if container preempted
+    cs.handle(new NodeUpdateSchedulerEvent(rmNode2));
+
+    FiCaSchedulerApp schedulerApp2 = cs.getApplicationAttempt(
+        am2.getApplicationAttemptId());
+
+    // App1 has 7 containers, and app2 has 1 containers (no container preempted)
+    Assert.assertEquals(7, schedulerApp1.getLiveContainers().size());
+    Assert.assertEquals(1, schedulerApp2.getLiveContainers().size());
+
+    // Do allocation again, nothing will be preempted
+    cs.handle(new NodeUpdateSchedulerEvent(rmNode2));
+
+    // App1 has 7 containers, and app2 has 1 containers (no container allocated)
+    Assert.assertEquals(7, schedulerApp1.getLiveContainers().size());
+    Assert.assertEquals(1, schedulerApp2.getLiveContainers().size());
+
+    rm1.close();
+  }
+
+  @Test (timeout = 60000)
+  public void testPreemptionPolicyShouldRespectAlreadyMarkedKillableContainers()
+      throws Exception {
+    /**
+     * Test case:
+     * <pre>
+     *             Root
+     *            /  |  \
+     *           a   b   c
+     *          10   20  70
+     * </pre>
+     * Submit applications to two queues, one uses more than the other, so
+     * preemption will happen.
+     *
+     * Check:
+     * 1) Killable containers resources will be excluded from PCPP (no duplicated
+     *    container added to killable list)
+     * 2) When more resources need to be preempted, new containers will be selected
+     *    and killable containers will be considered
+     */
+    MockRM rm1 = new MockRM(conf);
+    rm1.getRMContext().setNodeLabelManager(mgr);
+    rm1.start();
+    MockNM nm1 = rm1.registerNode("h1:1234", 8 * GB);
+    CapacityScheduler cs = (CapacityScheduler) rm1.getResourceScheduler();
+    RMNode rmNode1 = rm1.getRMContext().getRMNodes().get(nm1.getNodeId());
+
+    // launch an app to queue, AM container should be launched in nm1
+    RMApp app1 = rm1.submitApp(1 * GB, "app", "user", null, "a");
+    MockAM am1 = MockRM.launchAndRegisterAM(app1, rm1, nm1);
+
+    am1.allocate("*", 1 * GB, 6, new ArrayList<ContainerId>());
+
+    // Do allocation 6 times for node1
+    for (int i = 0; i < 6; i++) {
+      cs.handle(new NodeUpdateSchedulerEvent(rmNode1));
+    }
+
+    // App1 should have 7 containers now, and no available resource for cluster
+    FiCaSchedulerApp schedulerApp1 = cs.getApplicationAttempt(
+        am1.getApplicationAttemptId());
+    Assert.assertEquals(7, schedulerApp1.getLiveContainers().size());
+
+    // Submit app2 to queue-c and asks for a 1G container for AM
+    RMApp app2 = rm1.submitApp(1 * GB, "app", "user", null, "c");
+    MockAM am2 = MockRM.launchAndRegisterAM(app2, rm1, nm1);
+
+    // NM1 has available resource = 0G
+    Assert.assertEquals(0 * GB, cs.getNode(nm1.getNodeId())
+        .getUnallocatedResource().getMemory());
+    am2.allocate("*", 1 * GB, 1, new ArrayList<ContainerId>());
+
+    // Get edit policy and do one update
+    ProportionalCapacityPreemptionPolicy editPolicy =
+        (ProportionalCapacityPreemptionPolicy) getSchedulingEditPolicy(rm1);
+
+    // Call edit schedule twice, and check if one container from app1 marked
+    // to be "killable"
+    editPolicy.editSchedule();
+    editPolicy.editSchedule();
+
+    PreemptionManager pm = cs.getPreemptionManager();
+    waitKillableContainersSize(pm, "a", RMNodeLabelsManager.NO_LABEL, 1);
+
+    // Check killable containers and to-be-preempted containers in edit policy
+    Assert.assertEquals(0, editPolicy.getToPreemptContainers().size());
+
+    // Run edit schedule again, confirm status doesn't changed
+    editPolicy.editSchedule();
+    Assert.assertEquals(0, editPolicy.getToPreemptContainers().size());
+
+    // Save current to kill containers
+    Set<ContainerId> previousKillableContainers = new HashSet<>(
+        pm.getKillableContainersMap("a", RMNodeLabelsManager.NO_LABEL)
+            .keySet());
+
+    // Update request resource of c from 1 to 2, so we need to preempt
+    // one more container
+    am2.allocate("*", 1 * GB, 2, new ArrayList<ContainerId>());
+
+    // Call editPolicy.editSchedule() once, we should have 1 container in to-preempt map
+    // and 1 container in killable map
+    editPolicy.editSchedule();
+    Assert.assertEquals(1, editPolicy.getToPreemptContainers().size());
+
+    // Call editPolicy.editSchedule() once more, we should have 2 containers killable map
+    editPolicy.editSchedule();
+    Assert.assertEquals(0, editPolicy.getToPreemptContainers().size());
+
+    // Check if previous killable containers included by new killable containers
+    Map<ContainerId, RMContainer> killableContainers =
+        waitKillableContainersSize(pm, "a", RMNodeLabelsManager.NO_LABEL, 2);
+    Assert.assertTrue(
+        Sets.difference(previousKillableContainers, killableContainers.keySet())
+            .isEmpty());
+  }
+
+  @Test (timeout = 60000)
+  public void testPreemptionPolicyCleanupKillableContainersWhenNoPreemptionNeeded()
+      throws Exception {
+    /**
+     * Test case:
+     * <pre>
+     *             Root
+     *            /  |  \
+     *           a   b   c
+     *          10   20  70
+     * </pre>
+     * Submit applications to two queues, one uses more than the other, so
+     * preemption will happen.
+     *
+     * Check:
+     * 1) Containers will be marked to killable
+     * 2) Cancel resource request
+     * 3) Killable containers will be cancelled from policy and scheduler
+     */
+    MockRM rm1 = new MockRM(conf);
+    rm1.getRMContext().setNodeLabelManager(mgr);
+    rm1.start();
+    MockNM nm1 = rm1.registerNode("h1:1234", 8 * GB);
+    CapacityScheduler cs = (CapacityScheduler) rm1.getResourceScheduler();
+    RMNode rmNode1 = rm1.getRMContext().getRMNodes().get(nm1.getNodeId());
+
+    // launch an app to queue, AM container should be launched in nm1
+    RMApp app1 = rm1.submitApp(1 * GB, "app", "user", null, "a");
+    MockAM am1 = MockRM.launchAndRegisterAM(app1, rm1, nm1);
+
+    am1.allocate("*", 1 * GB, 6, new ArrayList<ContainerId>());
+
+    // Do allocation 6 times for node1
+    for (int i = 0; i < 6; i++) {
+      cs.handle(new NodeUpdateSchedulerEvent(rmNode1));
+    }
+
+    // App1 should have 7 containers now, and no available resource for cluster
+    FiCaSchedulerApp schedulerApp1 = cs.getApplicationAttempt(
+        am1.getApplicationAttemptId());
+    Assert.assertEquals(7, schedulerApp1.getLiveContainers().size());
+
+    // Submit app2 to queue-c and asks for a 1G container for AM
+    RMApp app2 = rm1.submitApp(1 * GB, "app", "user", null, "c");
+    MockAM am2 = MockRM.launchAndRegisterAM(app2, rm1, nm1);
+
+    // NM1 has available resource = 0G
+    Assert.assertEquals(0 * GB, cs.getNode(nm1.getNodeId())
+        .getUnallocatedResource().getMemory());
+    am2.allocate("*", 3 * GB, 1, new ArrayList<ContainerId>());
+
+    // Get edit policy and do one update
+    ProportionalCapacityPreemptionPolicy editPolicy =
+        (ProportionalCapacityPreemptionPolicy) getSchedulingEditPolicy(rm1);
+
+    // Call edit schedule twice, and check if 3 container from app1 marked
+    // to be "killable"
+    editPolicy.editSchedule();
+    editPolicy.editSchedule();
+
+    PreemptionManager pm = cs.getPreemptionManager();
+    waitKillableContainersSize(pm, "a", RMNodeLabelsManager.NO_LABEL, 3);
+
+    // Change reqeust from 3G to 2G, now we can preempt one less container. (3->2)
+    am2.allocate("*", 2 * GB, 1, new ArrayList<ContainerId>());
+    editPolicy.editSchedule();
+    Assert.assertEquals(0, editPolicy.getToPreemptContainers().size());
+    waitKillableContainersSize(pm, "a", RMNodeLabelsManager.NO_LABEL, 2);
+
+    // Call editSchedule once more to make sure still nothing happens
+    editPolicy.editSchedule();
+    Assert.assertEquals(0, editPolicy.getToPreemptContainers().size());
+    waitKillableContainersSize(pm, "a", RMNodeLabelsManager.NO_LABEL, 2);
+  }
+
+  @Test (timeout = 60000)
+  public void testPreemptionConsidersUserLimit()
+      throws Exception {
+    /**
+     * Test case: Submit two application (app1/app2) to different queues, queue
+     * structure:
+     *
+     * <pre>
+     *             Root
+     *            /  |  \
+     *           a   b   c
+     *          10   20  70
+     * </pre>
+     *
+     * Queue-c's user-limit-factor = 0.1, so single user cannot allocate >1 containers in queue-c
+     *
+     * 1) Two nodes in the cluster, each of them has 4G.
+     *
+     * 2) app1 submit to queue-a first, it asked 7 * 1G containers, so there's no
+     * more resource available.
+     *
+     * 3) app2 submit to queue-c, ask for one 1G container (for AM)
+     *
+     * Now the cluster is fulfilled.
+     *
+     * 4) app2 asks for another 1G container, system will preempt one container
+     * from app1, and app2 will receive the preempted container
+     */
+    CapacitySchedulerConfiguration csConf = new CapacitySchedulerConfiguration(conf);
+    csConf.setUserLimitFactor(CapacitySchedulerConfiguration.ROOT + ".c", 0.1f);
+    MockRM rm1 = new MockRM(csConf);
+    rm1.getRMContext().setNodeLabelManager(mgr);
+    rm1.start();
+    MockNM nm1 = rm1.registerNode("h1:1234", 4 * GB);
+    MockNM nm2 = rm1.registerNode("h2:1234", 4 * GB);
+    CapacityScheduler cs = (CapacityScheduler) rm1.getResourceScheduler();
+    RMNode rmNode1 = rm1.getRMContext().getRMNodes().get(nm1.getNodeId());
+    RMNode rmNode2 = rm1.getRMContext().getRMNodes().get(nm2.getNodeId());
+
+    // launch an app to queue, AM container should be launched in nm1
+    RMApp app1 = rm1.submitApp(1 * GB, "app", "user", null, "a");
+    MockAM am1 = MockRM.launchAndRegisterAM(app1, rm1, nm1);
+
+    am1.allocate("*", 1 * GB, 6, new ArrayList<ContainerId>());
+
+    // Do allocation 3 times for node1/node2
+    for (int i = 0; i < 3; i++) {
+      cs.handle(new NodeUpdateSchedulerEvent(rmNode1));
+      cs.handle(new NodeUpdateSchedulerEvent(rmNode2));
+    }
+
+    // App1 should have 7 containers now, and no available resource for cluster
+    FiCaSchedulerApp schedulerApp1 = cs.getApplicationAttempt(
+        am1.getApplicationAttemptId());
+    Assert.assertEquals(7, schedulerApp1.getLiveContainers().size());
+
+    // Submit app2 to queue-c and asks for a 1G container for AM
+    RMApp app2 = rm1.submitApp(1 * GB, "app", "user", null, "c");
+    MockAM am2 = MockRM.launchAndRegisterAM(app2, rm1, nm2);
+
+    // NM1/NM2 has available resource = 0G
+    Assert.assertEquals(0 * GB, cs.getNode(nm1.getNodeId())
+        .getUnallocatedResource().getMemory());
+    Assert.assertEquals(0 * GB, cs.getNode(nm2.getNodeId())
+        .getUnallocatedResource().getMemory());
+
+    // AM asks for a 1 * GB container
+    am2.allocate(Arrays.asList(ResourceRequest
+        .newInstance(Priority.newInstance(1), ResourceRequest.ANY,
+            Resources.createResource(1 * GB), 1)), null);
+
+    // Get edit policy and do one update
+    SchedulingEditPolicy editPolicy = getSchedulingEditPolicy(rm1);
+
+    // Call edit schedule twice, and check if no container from app1 marked
+    // to be "killable"
+    editPolicy.editSchedule();
+    editPolicy.editSchedule();
+
+    // No preemption happens
+    PreemptionManager pm = cs.getPreemptionManager();
+    Map<ContainerId, RMContainer> killableContainers =
+        waitKillableContainersSize(pm, "a", RMNodeLabelsManager.NO_LABEL, 0);
+    Assert.assertEquals(0, killableContainers.size());
+
+    // Call CS.handle once to see if container preempted
+    cs.handle(new NodeUpdateSchedulerEvent(rmNode2));
+
+    FiCaSchedulerApp schedulerApp2 = cs.getApplicationAttempt(
+        am2.getApplicationAttemptId());
+
+    // App1 has 7 containers, and app2 has 1 containers (nothing preempted)
+    Assert.assertEquals(7, schedulerApp1.getLiveContainers().size());
+    Assert.assertEquals(1, schedulerApp2.getLiveContainers().size());
+
+    rm1.close();
+  }
+
+  private Map<ContainerId, RMContainer> waitKillableContainersSize(
+      PreemptionManager pm, String queueName, String partition,
+      int expectedSize) throws InterruptedException {
+    Map<ContainerId, RMContainer> killableContainers =
+        pm.getKillableContainersMap(queueName, partition);
+
+    int wait = 0;
+    // Wait for at most 5 sec (it should be super fast actually)
+    while (expectedSize != killableContainers.size() && wait < 500) {
+      killableContainers = pm.getKillableContainersMap(queueName, partition);
+      Thread.sleep(10);
+      wait++;
+    }
+
+    Assert.assertEquals(expectedSize, killableContainers.size());
+    return killableContainers;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ae14e5d0/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 5169337..1612201 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
@@ -51,6 +51,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerEven
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerImpl;
 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.capacity.preemption.PreemptionManager;
 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.server.utils.BuilderUtils;
@@ -99,6 +100,7 @@ public class TestChildQueueOrder {
     when(csContext.getResourceCalculator()).
     thenReturn(resourceComparator);
     when(csContext.getRMContext()).thenReturn(rmContext);
+    when(csContext.getPreemptionManager()).thenReturn(new PreemptionManager());
   }
 
   private FiCaSchedulerApp getMockApplication(int appId, String user) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ae14e5d0/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 69b0813..87a3d51 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
@@ -71,6 +71,7 @@ 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.QueueMetrics;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceLimits;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.preemption.PreemptionManager;
 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.server.resourcemanager.scheduler.event.AppAddedSchedulerEvent;
@@ -150,6 +151,7 @@ public class TestLeafQueue {
         thenReturn(CapacityScheduler.nonPartitionedQueueComparator);
     when(csContext.getResourceCalculator()).
         thenReturn(resourceCalculator);
+    when(csContext.getPreemptionManager()).thenReturn(new PreemptionManager());
     when(csContext.getRMContext()).thenReturn(rmContext);
     RMContainerTokenSecretManager containerTokenSecretManager =
         new RMContainerTokenSecretManager(conf);
@@ -3092,6 +3094,7 @@ public class TestLeafQueue {
         Resources.createResource(GB, 1));
     when(csContext.getMaximumResourceCapability()).thenReturn(
         Resources.createResource(2 * GB, 2));
+    when(csContext.getPreemptionManager()).thenReturn(new PreemptionManager());
     return csContext;
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ae14e5d0/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 bbf6e43..1ee201d 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
@@ -39,6 +39,7 @@ 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.recovery.MemoryRMStateStore;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptState;
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
@@ -1676,4 +1677,100 @@ public class TestNodeLabelContainerAllocation {
     checkNumOfContainersInAnAppOnGivenNode(2, nm1.getNodeId(),
         cs.getApplicationAttempt(am1.getApplicationAttemptId()));
   }
+
+  @Test
+  public void testParentQueueMaxCapsAreRespected() throws Exception {
+    /*
+     * Queue tree:
+     *          Root
+     *        /     \
+     *       A       B
+     *      / \
+     *     A1 A2
+     *
+     * A has 50% capacity and 50% max capacity (of label=x)
+     * A1/A2 has 50% capacity and 100% max capacity (of label=x)
+     * Cluster has one node (label=x) with resource = 24G.
+     * So we can at most use 12G resources under queueA.
+     */
+    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, 10);
+    csConf.setAccessibleNodeLabels(A, toSet("x"));
+    csConf.setCapacityByLabel(A, "x", 50);
+    csConf.setMaximumCapacityByLabel(A, "x", 50);
+
+    final String B = CapacitySchedulerConfiguration.ROOT + ".b";
+    csConf.setCapacity(B, 90);
+    csConf.setAccessibleNodeLabels(B, toSet("x"));
+    csConf.setCapacityByLabel(B, "x", 50);
+    csConf.setMaximumCapacityByLabel(B, "x", 50);
+
+    // Define 2nd-level queues
+    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);
+    csConf.setMaximumCapacityByLabel(A1, "x", 100);
+    csConf.setUserLimitFactor(A1, 100.0f);
+
+    final String A2 = A + ".a2";
+    csConf.setCapacity(A2, 50);
+    csConf.setAccessibleNodeLabels(A2, toSet("x"));
+    csConf.setCapacityByLabel(A2, "x", 50);
+    csConf.setMaximumCapacityByLabel(A2, "x", 100);
+    csConf.setUserLimitFactor(A2, 100.0f);
+
+    // set node -> label
+    mgr.addToCluserNodeLabels(ImmutableSet.of(
+        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 =
+        new MockNM("h1:1234", 24 * GB, rm.getResourceTrackerService());
+    nm1.registerNode();
+
+    // Launch app1 in a1, resource usage is 1GB (am) + 4GB * 2 = 9GB
+    RMApp app1 = rm.submitApp(1 * GB, "app", "user", null, "a1", "x");
+    MockAM am1 = MockRM.launchAndRegisterAM(app1, rm, nm1);
+    am1.allocate("*", 4 * GB, 2, new ArrayList<ContainerId>(), "x");
+    doNMHeartbeat(rm, nm1.getNodeId(), 10);
+    checkNumOfContainersInAnAppOnGivenNode(3, nm1.getNodeId(),
+        cs.getApplicationAttempt(am1.getApplicationAttemptId()));
+
+    // Try to launch app2 in a2, asked 2GB, should success
+    RMApp app2 = rm.submitApp(2 * GB, "app", "user", null, "a2", "x");
+    MockAM am2 = MockRM.launchAndRegisterAM(app2, rm, nm1);
+
+    // am2 asks more resources, cannot success because current used = 9G (app1)
+    // + 2G (app2) = 11G, and queue's max capacity = 12G
+    am2.allocate("*", 2 * GB, 2, new ArrayList<ContainerId>(), "x");
+
+    doNMHeartbeat(rm, nm1.getNodeId(), 10);
+    checkNumOfContainersInAnAppOnGivenNode(1, nm1.getNodeId(),
+        cs.getApplicationAttempt(am2.getApplicationAttemptId()));
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ae14e5d0/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 f73baa4..23dc860 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
@@ -47,6 +47,7 @@ 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.capacity.preemption.PreemptionManager;
 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.DefaultResourceCalculator;
@@ -92,6 +93,7 @@ public class TestParentQueue {
         thenReturn(Resources.createResource(100 * 16 * GB, 100 * 32));
     when(csContext.getNonPartitionedQueueComparator()).
     thenReturn(CapacityScheduler.nonPartitionedQueueComparator);
+    when(csContext.getPreemptionManager()).thenReturn(new PreemptionManager());
     when(csContext.getResourceCalculator()).
     thenReturn(resourceComparator);
     when(csContext.getRMContext()).thenReturn(rmContext);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ae14e5d0/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 2ef5e39..56facee 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
@@ -55,6 +55,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerImpl
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ActiveUsersManager;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceLimits;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplicationAttempt.AMState;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.preemption.PreemptionManager;
 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.server.resourcemanager.security.RMContainerTokenSecretManager;
@@ -126,6 +127,7 @@ public class TestReservations {
     when(csContext.getNonPartitionedQueueComparator()).thenReturn(
         CapacityScheduler.nonPartitionedQueueComparator);
     when(csContext.getResourceCalculator()).thenReturn(resourceCalculator);
+    when(csContext.getPreemptionManager()).thenReturn(new PreemptionManager());
     when(csContext.getRMContext()).thenReturn(rmContext);
     RMContainerTokenSecretManager containerTokenSecretManager = new RMContainerTokenSecretManager(
         conf);


[35/46] hadoop git commit: YARN-4108. CapacityScheduler: Improve preemption to only kill containers that would satisfy the incoming request. (Wangda Tan)

Posted by ae...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/ae14e5d0/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 2694957..4441c6b 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
@@ -356,4 +356,40 @@ public class TestUtils {
     CapacityScheduler cs = (CapacityScheduler) rm.getResourceScheduler();
     return cs.getSchedulerApplications().get(appId).getCurrentAppAttempt();
   }
+
+  /**
+   * Get a queue structure:
+   * <pre>
+   *             Root
+   *            /  |  \
+   *           a   b   c
+   *          10   20  70
+   * </pre>
+   */
+  public static Configuration
+  getConfigurationWithMultipleQueues(Configuration config) {
+    CapacitySchedulerConfiguration conf =
+        new CapacitySchedulerConfiguration(config);
+
+    // Define top-level queues
+    conf.setQueues(CapacitySchedulerConfiguration.ROOT,
+        new String[] { "a", "b", "c" });
+
+    final String A = CapacitySchedulerConfiguration.ROOT + ".a";
+    conf.setCapacity(A, 10);
+    conf.setMaximumCapacity(A, 100);
+    conf.setUserLimitFactor(A, 100);
+
+    final String B = CapacitySchedulerConfiguration.ROOT + ".b";
+    conf.setCapacity(B, 20);
+    conf.setMaximumCapacity(B, 100);
+    conf.setUserLimitFactor(B, 100);
+
+    final String C = CapacitySchedulerConfiguration.ROOT + ".c";
+    conf.setCapacity(C, 70);
+    conf.setMaximumCapacity(C, 100);
+    conf.setUserLimitFactor(C, 100);
+
+    return conf;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ae14e5d0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFairSchedulerPreemption.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/fair/TestFairSchedulerPreemption.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFairSchedulerPreemption.java
index 5bdcc08..2456594 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFairSchedulerPreemption.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFairSchedulerPreemption.java
@@ -1451,7 +1451,7 @@ public class TestFairSchedulerPreemption extends FairSchedulerTestBase {
 
     // Trigger container rescheduled event
     scheduler.handle(new ContainerPreemptEvent(appAttemptId, rmContainer,
-            SchedulerEventType.KILL_PREEMPTED_CONTAINER));
+            SchedulerEventType.MARK_CONTAINER_FOR_KILLABLE));
 
     List<ResourceRequest> requests = rmContainer.getResourceRequests();
     // Once recovered, resource request will be present again in app


[44/46] hadoop git commit: HDFS-3677. dfs.namenode.edits.dir.required is missing from hdfs-default.xml. Contributed by Mark Yang.

Posted by ae...@apache.org.
HDFS-3677. dfs.namenode.edits.dir.required is missing from hdfs-default.xml. Contributed by Mark Yang.


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

Branch: refs/heads/HDFS-7240
Commit: 9b623fbaf79c0f854abc6b4a0539d4ea8c93dc1a
Parents: 63c966a
Author: Akira Ajisaka <aa...@apache.org>
Authored: Sat Mar 19 01:09:02 2016 +0900
Committer: Akira Ajisaka <aa...@apache.org>
Committed: Sat Mar 19 01:11:15 2016 +0900

----------------------------------------------------------------------
 .../hadoop-hdfs/src/main/resources/hdfs-default.xml         | 9 +++++++++
 1 file changed, 9 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/9b623fba/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml b/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
index d837bd8..d1f78ca 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
@@ -384,6 +384,15 @@
 </property>
 
 <property>
+  <name>dfs.namenode.edits.dir.required</name>
+  <value></value>
+  <description>This should be a subset of dfs.namenode.edits.dir,
+      to ensure that the transaction (edits) file
+      in these places is always up-to-date.
+  </description>
+</property>
+
+<property>
   <name>dfs.namenode.shared.edits.dir</name>
   <value></value>
   <description>A directory on shared storage between the multiple namenodes


[18/46] hadoop git commit: HDFS-9904. testCheckpointCancellationDuringUpload occasionally fails. Contributed by Lin Yiqun.

Posted by ae...@apache.org.
HDFS-9904. testCheckpointCancellationDuringUpload occasionally fails. Contributed by Lin Yiqun.


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

Branch: refs/heads/HDFS-7240
Commit: d4574017845cfa7521e703f80efd404afd09b8c4
Parents: 5de848c
Author: Kihwal Lee <ki...@apache.org>
Authored: Tue Mar 15 10:52:47 2016 -0500
Committer: Kihwal Lee <ki...@apache.org>
Committed: Tue Mar 15 10:52:47 2016 -0500

----------------------------------------------------------------------
 .../hadoop/hdfs/server/namenode/ha/TestStandbyCheckpoints.java  | 5 +++++
 1 file changed, 5 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/d4574017/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestStandbyCheckpoints.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestStandbyCheckpoints.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestStandbyCheckpoints.java
index 234bc7b..7c0ed7b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestStandbyCheckpoints.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestStandbyCheckpoints.java
@@ -291,6 +291,11 @@ public class TestStandbyCheckpoints {
    */
   @Test(timeout=60000)
   public void testCheckpointCancellationDuringUpload() throws Exception {
+    // Set dfs.namenode.checkpoint.txns differently on the first NN to avoid it
+    // doing checkpoint when it becomes a standby
+    cluster.getConfiguration(0).setInt(
+        DFSConfigKeys.DFS_NAMENODE_CHECKPOINT_TXNS_KEY, 1000);
+
     // don't compress, we want a big image
     for (int i = 0; i < NUM_NNS; i++) {
       cluster.getConfiguration(i).setBoolean(


[46/46] hadoop git commit: Merge branch 'trunk' into HDFS-7240

Posted by ae...@apache.org.
Merge branch 'trunk' into HDFS-7240


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

Branch: refs/heads/HDFS-7240
Commit: 37e3a36a3f3ab217afc47f1120db6a4f03559efb
Parents: 4143f03 92b7e0d
Author: Anu Engineer <ae...@apache.org>
Authored: Fri Mar 18 11:18:48 2016 -0700
Committer: Anu Engineer <ae...@apache.org>
Committed: Fri Mar 18 11:18:48 2016 -0700

----------------------------------------------------------------------
 .../hadoop-common/src/CMakeLists.txt            |  22 +-
 .../apache/hadoop/io/erasurecode/CodecUtil.java |  13 +-
 .../io/erasurecode/rawcoder/RSRawDecoder.java   | 175 +++++
 .../io/erasurecode/rawcoder/RSRawDecoder2.java  | 176 -----
 .../io/erasurecode/rawcoder/RSRawEncoder.java   |  75 ++
 .../io/erasurecode/rawcoder/RSRawEncoder2.java  |  76 ---
 .../rawcoder/RSRawErasureCoderFactory.java      |  37 +
 .../rawcoder/RSRawErasureCoderFactory2.java     |  37 -
 .../io/erasurecode/rawcoder/util/RSUtil.java    | 149 +++-
 .../io/erasurecode/rawcoder/util/RSUtil2.java   | 172 -----
 .../main/java/org/apache/hadoop/ipc/Client.java | 109 +--
 .../apache/hadoop/ipc/WritableRpcEngine.java    |   2 +-
 .../java/org/apache/hadoop/security/Groups.java |  17 +-
 .../apache/hadoop/service/AbstractService.java  |   2 +-
 .../main/java/org/apache/hadoop/util/Shell.java |  10 +-
 .../src/main/native/native.vcxproj              |  11 +-
 .../io/erasurecode/coder/erasure_code_native.c  |  49 --
 ...he_hadoop_io_erasurecode_ErasureCodeNative.h |  29 -
 .../src/org/apache/hadoop/io/erasurecode/dump.c | 100 +++
 .../src/org/apache/hadoop/io/erasurecode/dump.h |  40 ++
 .../apache/hadoop/io/erasurecode/erasure_code.c | 239 +------
 .../apache/hadoop/io/erasurecode/erasure_code.h | 107 +++
 .../hadoop/io/erasurecode/erasure_coder.c       | 229 +++++++
 .../hadoop/io/erasurecode/erasure_coder.h       |  88 +++
 .../org/apache/hadoop/io/erasurecode/gf_util.c  |  54 ++
 .../org/apache/hadoop/io/erasurecode/gf_util.h  | 111 +++
 .../io/erasurecode/include/erasure_code.h       | 125 ----
 .../hadoop/io/erasurecode/include/gf_util.h     | 111 ---
 .../apache/hadoop/io/erasurecode/isal_load.c    | 148 ++++
 .../apache/hadoop/io/erasurecode/isal_load.h    | 149 ++++
 .../apache/hadoop/io/erasurecode/jni_common.c   |  98 +++
 .../apache/hadoop/io/erasurecode/jni_common.h   |  42 ++
 .../io/erasurecode/jni_erasure_code_native.c    |  45 ++
 .../hadoop/io/erasurecode/jni_rs_decoder.c      |  72 ++
 .../hadoop/io/erasurecode/jni_rs_encoder.c      |  66 ++
 ...he_hadoop_io_erasurecode_ErasureCodeNative.h |  29 +
 ...io_erasurecode_rawcoder_NativeRSRawDecoder.h |  37 +
 ...io_erasurecode_rawcoder_NativeRSRawEncoder.h |  37 +
 .../hadoop/io/erasurecode/erasure_code_test.c   | 307 ++-------
 .../coder/TestHHXORErasureCoder.java            |   4 +-
 .../erasurecode/coder/TestRSErasureCoder.java   |   8 +-
 .../io/erasurecode/rawcoder/TestRSRawCoder.java |  33 +
 .../erasurecode/rawcoder/TestRSRawCoder2.java   |  33 -
 .../java/org/apache/hadoop/ipc/TestIPC.java     | 101 +--
 .../hadoop/ipc/TestIPCServerResponder.java      |  30 +-
 .../java/org/apache/hadoop/ipc/TestSaslRPC.java |  18 +-
 .../org/apache/hadoop/hdfs/protocol/Block.java  |   4 +-
 .../server/blockmanagement/BlockManager.java    | 268 ++++----
 .../BlockUnderConstructionFeature.java          |  45 +-
 .../blockmanagement/DecommissionManager.java    |  38 +-
 .../blockmanagement/LowRedundancyBlocks.java    | 458 +++++++++++++
 .../blockmanagement/UnderReplicatedBlocks.java  | 448 ------------
 .../hdfs/server/datanode/DataXceiverServer.java |   2 +-
 .../hdfs/server/datanode/ReplicaInPipeline.java |   7 +
 .../datanode/fsdataset/impl/FsDatasetImpl.java  |  13 +
 .../datanode/fsdataset/impl/FsVolumeImpl.java   |   6 +
 .../hdfs/server/namenode/FSDirWriteFileOp.java  |  18 +-
 .../hdfs/server/namenode/NameNodeRpcServer.java |   4 +
 .../src/main/resources/hdfs-default.xml         |   9 +
 .../src/main/webapps/hdfs/explorer.js           |  33 +-
 .../src/site/markdown/HDFSCommands.md           | 159 +++--
 .../blockmanagement/BlockManagerTestUtil.java   |   2 +-
 .../blockmanagement/TestBlockManager.java       |  20 +-
 .../TestLowRedundancyBlockQueues.java           | 182 +++++
 .../blockmanagement/TestPendingReplication.java |  14 +-
 .../blockmanagement/TestReplicationPolicy.java  | 158 ++---
 .../TestUnderReplicatedBlockQueues.java         | 182 -----
 .../hdfs/server/datanode/TestDataNodeUUID.java  |  52 ++
 .../fsdataset/impl/TestFsDatasetImpl.java       |  66 ++
 .../hdfs/server/namenode/TestMetaSave.java      |   2 +-
 .../namenode/ha/TestStandbyCheckpoints.java     |   5 +
 .../org/apache/hadoop/mapred/ClusterStatus.java |  35 +-
 .../org/apache/hadoop/mapred/NotRunningJob.java |   7 +-
 .../hadoop/conf/TestNoDefaultsJobConf.java      |  30 +-
 .../apache/hadoop/mapred/HadoopTestCase.java    |  17 +-
 .../hadoop/mapred/NotificationTestCase.java     |  15 +-
 .../hadoop/mapred/TestFileOutputFormat.java     |   4 +
 .../apache/hadoop/mapred/TestNetworkedJob.java  |  89 ++-
 .../apache/hadoop/mapred/TestTaskCommit.java    |  31 +-
 .../mapred/jobcontrol/TestLocalJobControl.java  |   3 +
 .../hadoop/mapred/lib/TestChainMapReduce.java   |  20 +-
 .../mapred/lib/TestKeyFieldBasedComparator.java |   2 +
 .../hadoop/mapred/lib/TestMultipleInputs.java   |  17 +-
 .../hadoop/mapred/lib/TestMultipleOutputs.java  |  26 +-
 .../mapred/lib/TestMultithreadedMapRunner.java  |  21 +-
 .../org/apache/hadoop/mapreduce/TestChild.java  |  10 +-
 .../hadoop/mapreduce/TestNoJobSetupCleanup.java |   6 +-
 .../hadoop/mapreduce/TestTaskContext.java       |   2 +
 .../mapreduce/lib/chain/TestChainErrors.java    |   9 +
 .../mapreduce/lib/chain/TestMapReduceChain.java |   6 +
 .../lib/chain/TestSingleElementChain.java       |   4 +
 .../lib/db/TestDataDrivenDBInputFormat.java     |  40 +-
 .../mapreduce/lib/input/TestMultipleInputs.java |   6 +-
 .../lib/jobcontrol/TestMapReduceJobControl.java |   8 +-
 .../lib/map/TestMultithreadedMapper.java        |  12 +-
 .../lib/output/TestJobOutputCommitter.java      |  16 +-
 .../lib/output/TestMRMultipleOutputs.java       |  17 +-
 .../TestMRKeyFieldBasedComparator.java          |  20 +-
 .../apache/hadoop/examples/TestWordStats.java   |  15 +-
 .../hadoop/examples/terasort/TestTeraSort.java  |  11 +-
 hadoop-project/pom.xml                          |  13 +
 .../pom.xml                                     |  21 +
 .../distributedshell/ApplicationMaster.java     |  68 +-
 .../DistributedShellTimelinePlugin.java         |  79 +++
 .../distributedshell/package-info.java          |  19 +
 .../distributedshell/TestDistributedShell.java  | 120 +++-
 .../hadoop/yarn/client/ProtocolHATestBase.java  |   3 +-
 .../hadoop/yarn/client/TestRMFailover.java      |   2 -
 .../yarn/client/api/impl/TestYarnClient.java    |  18 +
 .../api/impl/FileSystemTimelineWriter.java      |   6 +-
 .../apache/hadoop/yarn/util/SystemClock.java    |   5 +-
 .../yarn/util/timeline/TimelineUtils.java       |  35 +
 .../nodemanager/NodeStatusUpdaterImpl.java      |  50 +-
 .../TestResourceLocalizationService.java        |   2 +-
 .../ProportionalCapacityPreemptionPolicy.java   | 166 +++--
 .../rmcontainer/RMContainer.java                |   1 +
 .../scheduler/AbstractYarnScheduler.java        | 170 +----
 .../scheduler/ClusterNodeTracker.java           | 300 ++++++++
 .../resourcemanager/scheduler/NodeFilter.java   |  33 +
 .../scheduler/PreemptableResourceScheduler.java |   2 +-
 .../scheduler/ResourceLimits.java               |   9 +
 .../scheduler/SchedulerNode.java                |   9 +-
 .../scheduler/capacity/AbstractCSQueue.java     |  45 +-
 .../scheduler/capacity/CSAssignment.java        |  11 +
 .../scheduler/capacity/CapacityScheduler.java   | 216 +++---
 .../CapacitySchedulerConfiguration.java         |  14 +-
 .../capacity/CapacitySchedulerContext.java      |  15 +-
 .../scheduler/capacity/LeafQueue.java           |  69 ++
 .../scheduler/capacity/ParentQueue.java         | 157 ++++-
 .../allocator/AbstractContainerAllocator.java   |   2 +
 .../capacity/allocator/ContainerAllocation.java |  12 +
 .../allocator/RegularContainerAllocator.java    |  39 +-
 .../capacity/preemption/KillableContainer.java  |  45 ++
 .../capacity/preemption/PreemptableQueue.java   | 102 +++
 .../capacity/preemption/PreemptionManager.java  | 165 +++++
 .../scheduler/common/AssignmentInformation.java |   6 +-
 .../scheduler/common/fica/FiCaSchedulerApp.java |  32 +-
 .../common/fica/FiCaSchedulerNode.java          |  65 +-
 .../scheduler/event/SchedulerEventType.java     |  15 +-
 .../scheduler/fair/FSAppAttempt.java            |  13 +-
 .../scheduler/fair/FairScheduler.java           | 122 ++--
 .../scheduler/fifo/FifoScheduler.java           |  32 +-
 .../webapp/dao/CapacitySchedulerInfo.java       |  26 +-
 .../resourcemanager/TestRMDispatcher.java       |   4 +-
 .../server/resourcemanager/TestRMRestart.java   |   2 +-
 .../applicationsmanager/TestAMRestart.java      |   7 +-
 ...estProportionalCapacityPreemptionPolicy.java |   6 +-
 ...pacityPreemptionPolicyForNodePartitions.java |   2 +
 .../scheduler/TestAbstractYarnScheduler.java    |  14 +-
 .../capacity/TestApplicationLimits.java         |   2 +
 .../capacity/TestApplicationPriority.java       |   6 +-
 .../capacity/TestCapacityScheduler.java         |   8 +-
 .../TestCapacitySchedulerPreemption.java        | 677 +++++++++++++++++++
 .../scheduler/capacity/TestChildQueueOrder.java |   2 +
 .../scheduler/capacity/TestLeafQueue.java       |   3 +
 .../TestNodeLabelContainerAllocation.java       |  97 +++
 .../scheduler/capacity/TestParentQueue.java     |   2 +
 .../scheduler/capacity/TestReservations.java    |  21 +-
 .../scheduler/capacity/TestUtils.java           |  36 +
 .../scheduler/fair/FairSchedulerTestBase.java   |  22 +-
 .../fair/TestContinuousScheduling.java          |  78 ++-
 .../scheduler/fair/TestFairScheduler.java       |  82 +--
 .../fair/TestFairSchedulerPreemption.java       |   2 +-
 .../scheduler/fifo/TestFifoScheduler.java       |  19 +-
 .../webapp/TestRMWebServicesCapacitySched.java  |   3 +
 .../hadoop/yarn/server/MiniYARNCluster.java     |  85 +--
 .../yarn/server/TestMiniYARNClusterForHA.java   |   4 -
 .../yarn/server/timeline/TimelineVersion.java   |  31 +
 .../server/timeline/TimelineVersionWatcher.java |  47 ++
 .../pom.xml                                     |  16 +
 .../server/timeline/PluginStoreTestUtils.java   |  51 +-
 171 files changed, 6693 insertions(+), 3286 deletions(-)
----------------------------------------------------------------------



[04/46] hadoop git commit: MAPREDUCE-6520. Migrate MR Client test cases part 1. Contributed by Dustin Cote.

Posted by ae...@apache.org.
MAPREDUCE-6520. Migrate MR Client test cases part 1. Contributed by Dustin Cote.


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

Branch: refs/heads/HDFS-7240
Commit: 247a7906092065289ea81139e71badcac6abef1e
Parents: 6876b9f
Author: Akira Ajisaka <aa...@apache.org>
Authored: Fri Mar 11 22:51:20 2016 +0900
Committer: Akira Ajisaka <aa...@apache.org>
Committed: Fri Mar 11 22:51:20 2016 +0900

----------------------------------------------------------------------
 .../hadoop/conf/TestNoDefaultsJobConf.java      | 30 +++++++++++----
 .../apache/hadoop/mapred/HadoopTestCase.java    | 17 ++++-----
 .../hadoop/mapred/NotificationTestCase.java     | 15 ++++++--
 .../hadoop/mapred/TestFileOutputFormat.java     |  4 ++
 .../apache/hadoop/mapred/TestTaskCommit.java    | 31 ++++++++++-----
 .../mapred/jobcontrol/TestLocalJobControl.java  |  3 ++
 .../hadoop/mapred/lib/TestChainMapReduce.java   | 20 +++++++++-
 .../mapred/lib/TestKeyFieldBasedComparator.java |  2 +
 .../hadoop/mapred/lib/TestMultipleInputs.java   | 17 +++++----
 .../hadoop/mapred/lib/TestMultipleOutputs.java  | 26 ++++++++++++-
 .../mapred/lib/TestMultithreadedMapRunner.java  | 21 +++++++++-
 .../org/apache/hadoop/mapreduce/TestChild.java  | 10 ++++-
 .../hadoop/mapreduce/TestNoJobSetupCleanup.java |  6 ++-
 .../hadoop/mapreduce/TestTaskContext.java       |  2 +
 .../mapreduce/lib/chain/TestChainErrors.java    |  9 +++++
 .../mapreduce/lib/chain/TestMapReduceChain.java |  6 +++
 .../lib/chain/TestSingleElementChain.java       |  4 ++
 .../lib/db/TestDataDrivenDBInputFormat.java     | 40 ++++++++++++++------
 .../mapreduce/lib/input/TestMultipleInputs.java |  6 ++-
 .../lib/jobcontrol/TestMapReduceJobControl.java |  8 +++-
 .../lib/map/TestMultithreadedMapper.java        | 12 +++++-
 .../lib/output/TestJobOutputCommitter.java      | 16 ++++++--
 .../lib/output/TestMRMultipleOutputs.java       | 17 ++++++++-
 .../TestMRKeyFieldBasedComparator.java          | 20 +++++++---
 .../hadoop/examples/terasort/TestTeraSort.java  | 11 +++++-
 25 files changed, 285 insertions(+), 68 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/247a7906/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/conf/TestNoDefaultsJobConf.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/conf/TestNoDefaultsJobConf.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/conf/TestNoDefaultsJobConf.java
index b47d5ef..e2d75ab 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/conf/TestNoDefaultsJobConf.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/conf/TestNoDefaultsJobConf.java
@@ -17,17 +17,30 @@
  */
 package org.apache.hadoop.conf;
 
-import org.junit.Assert;
-
-import org.apache.hadoop.mapred.*;
-import org.apache.hadoop.mapreduce.MRConfig;
-import org.apache.hadoop.mapreduce.server.jobtracker.JTConfig;
-import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.FileUtil;
+import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.LongWritable;
 import org.apache.hadoop.io.Text;
-
-import java.io.*;
+import org.apache.hadoop.mapred.FileInputFormat;
+import org.apache.hadoop.mapred.FileOutputFormat;
+import org.apache.hadoop.mapred.HadoopTestCase;
+import org.apache.hadoop.mapred.JobClient;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.TextInputFormat;
+import org.apache.hadoop.mapred.TextOutputFormat;
+import org.apache.hadoop.mapred.Utils;
+import org.junit.Test;
+
+import java.io.BufferedReader;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.io.OutputStream;
+import java.io.OutputStreamWriter;
+import java.io.Writer;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
 
 /**
  * This testcase tests that a JobConf without default values submits jobs
@@ -40,6 +53,7 @@ public class TestNoDefaultsJobConf extends HadoopTestCase {
     super(HadoopTestCase.CLUSTER_MR, HadoopTestCase.DFS_FS, 1, 1);
   }
 
+  @Test
   public void testNoDefaults() throws Exception {
     JobConf configuration = new JobConf();
     assertTrue(configuration.get("hadoop.tmp.dir", null) != null);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/247a7906/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/HadoopTestCase.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/HadoopTestCase.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/HadoopTestCase.java
index 3cd0668..277c0fd 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/HadoopTestCase.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/HadoopTestCase.java
@@ -18,13 +18,12 @@
 
 package org.apache.hadoop.mapred;
 
-import junit.framework.TestCase;
-import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.mapreduce.MRConfig;
+import org.junit.After;
+import org.junit.Before;
 
-import java.io.File;
 import java.io.IOException;
 
 /**
@@ -38,7 +37,7 @@ import java.io.IOException;
  * Job Configurations should be created using a configuration returned by the
  * 'createJobConf()' method.
  */
-public abstract class HadoopTestCase extends TestCase {
+public abstract class HadoopTestCase {
   public static final int LOCAL_MR = 1;
   public static final int CLUSTER_MR = 2;
   public static final int LOCAL_FS = 4;
@@ -140,8 +139,8 @@ public abstract class HadoopTestCase extends TestCase {
    *
    * @throws Exception
    */
-  protected void setUp() throws Exception {
-    super.setUp();
+  @Before
+  public void setUp() throws Exception {
     if (localFS) {
       fileSystem = FileSystem.getLocal(new JobConf());
     }
@@ -164,7 +163,8 @@ public abstract class HadoopTestCase extends TestCase {
    *
    * @throws Exception
    */
-  protected void tearDown() throws Exception {
+  @After
+  public void tearDown() throws Exception {
     try {
       if (mrCluster != null) {
         mrCluster.shutdown();
@@ -181,7 +181,6 @@ public abstract class HadoopTestCase extends TestCase {
     catch (Exception ex) {
       System.out.println(ex);
     }
-    super.tearDown();
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/247a7906/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/NotificationTestCase.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/NotificationTestCase.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/NotificationTestCase.java
index d2ea74e..1f657cf 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/NotificationTestCase.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/NotificationTestCase.java
@@ -34,6 +34,13 @@ import javax.servlet.ServletException;
 import java.io.IOException;
 import java.io.DataOutputStream;
 
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.assertEquals;
+import org.junit.Before;
+import org.junit.After;
+import org.junit.Test;
+
+
 /**
  * Base class to test Job end notification in local and cluster mode.
  *
@@ -140,17 +147,19 @@ public abstract class NotificationTestCase extends HadoopTestCase {
     return conf;
   }
 
-
-  protected void setUp() throws Exception {
+  @Before
+  public void setUp() throws Exception {
     super.setUp();
     startHttpServer();
   }
 
-  protected void tearDown() throws Exception {
+  @After
+  public void tearDown() throws Exception {
     stopHttpServer();
     super.tearDown();
   }
 
+  @Test
   public void testMR() throws Exception {
 
     System.out.println(launchWordCount(this.createJobConf(),

http://git-wip-us.apache.org/repos/asf/hadoop/blob/247a7906/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestFileOutputFormat.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestFileOutputFormat.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestFileOutputFormat.java
index 81b53cc..3141235 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestFileOutputFormat.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestFileOutputFormat.java
@@ -30,12 +30,16 @@ import java.io.IOException;
 import java.io.OutputStream;
 import java.util.Iterator;
 
+import org.junit.Test;
+import static org.junit.Assert.assertTrue;
+
 public class TestFileOutputFormat extends HadoopTestCase {
 
   public TestFileOutputFormat() throws IOException {
     super(HadoopTestCase.LOCAL_MR, HadoopTestCase.LOCAL_FS, 1, 1);
   }
 
+  @Test
   public void testCustomFile() throws Exception {
     Path inDir = new Path("testing/fileoutputformat/input");
     Path outDir = new Path("testing/fileoutputformat/output");

http://git-wip-us.apache.org/repos/asf/hadoop/blob/247a7906/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestTaskCommit.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestTaskCommit.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestTaskCommit.java
index bad06e9..bed545e 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestTaskCommit.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestTaskCommit.java
@@ -17,9 +17,6 @@
  */
 package org.apache.hadoop.mapred;
 
-import java.io.File;
-import java.io.IOException;
-
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.Path;
@@ -27,9 +24,18 @@ import org.apache.hadoop.io.Text;
 import org.apache.hadoop.ipc.ProtocolSignature;
 import org.apache.hadoop.mapred.SortedRanges.Range;
 import org.apache.hadoop.mapreduce.TaskType;
-import org.apache.hadoop.mapreduce.checkpoint.CheckpointID;
-import org.apache.hadoop.mapreduce.checkpoint.FSCheckpointID;
 import org.apache.hadoop.mapreduce.checkpoint.TaskCheckpointID;
+import org.junit.After;
+import org.junit.Test;
+
+import java.io.File;
+import java.io.IOException;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
 
 
 public class TestTaskCommit extends HadoopTestCase {
@@ -80,12 +86,13 @@ public class TestTaskCommit extends HadoopTestCase {
     super(LOCAL_MR, LOCAL_FS, 1, 1);
   }
   
-  @Override
+  @After
   public void tearDown() throws Exception {
     super.tearDown();
     FileUtil.fullyDelete(new File(rootDir.toString()));
   }
-  
+
+  @Test
   public void testCommitFail() throws IOException {
     final Path inDir = new Path(rootDir, "./input");
     final Path outDir = new Path(rootDir, "./output");
@@ -199,6 +206,7 @@ public class TestTaskCommit extends HadoopTestCase {
    * 
    * @throws Exception
    */
+  @Test
   public void testTaskCleanupDoesNotCommit() throws Exception {
     // Mimic a job with a special committer that does not cleanup
     // files when a task fails.
@@ -245,23 +253,27 @@ public class TestTaskCommit extends HadoopTestCase {
     assertTrue("Task did not succeed", umbilical.taskDone);
   }
 
+  @Test
   public void testCommitRequiredForMapTask() throws Exception {
     Task testTask = createDummyTask(TaskType.MAP);
     assertTrue("MapTask should need commit", testTask.isCommitRequired());
   }
 
+  @Test
   public void testCommitRequiredForReduceTask() throws Exception {
     Task testTask = createDummyTask(TaskType.REDUCE);
     assertTrue("ReduceTask should need commit", testTask.isCommitRequired());
   }
-  
+
+  @Test
   public void testCommitNotRequiredForJobSetup() throws Exception {
     Task testTask = createDummyTask(TaskType.MAP);
     testTask.setJobSetupTask();
     assertFalse("Job setup task should not need commit", 
         testTask.isCommitRequired());
   }
-  
+
+  @Test
   public void testCommitNotRequiredForJobCleanup() throws Exception {
     Task testTask = createDummyTask(TaskType.MAP);
     testTask.setJobCleanupTask();
@@ -269,6 +281,7 @@ public class TestTaskCommit extends HadoopTestCase {
         testTask.isCommitRequired());
   }
 
+  @Test
   public void testCommitNotRequiredForTaskCleanup() throws Exception {
     Task testTask = createDummyTask(TaskType.REDUCE);
     testTask.setTaskCleanupTask();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/247a7906/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/jobcontrol/TestLocalJobControl.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/jobcontrol/TestLocalJobControl.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/jobcontrol/TestLocalJobControl.java
index 8d35dcf..07b1306 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/jobcontrol/TestLocalJobControl.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/jobcontrol/TestLocalJobControl.java
@@ -27,6 +27,8 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.mapred.HadoopTestCase;
 import org.apache.hadoop.mapred.JobConf;
+import org.junit.Test;
+import static org.junit.Assert.assertEquals;
 
 /**
  * HadoopTestCase that tests the local job runner.
@@ -59,6 +61,7 @@ public class TestLocalJobControl extends HadoopTestCase {
    * object. Finally, it creates a thread to run the JobControl object and
    * monitors/reports the job states.
    */
+  @Test
   public void testLocalJobControlDataCopy() throws Exception {
 
     FileSystem fs = FileSystem.get(createJobConf());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/247a7906/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/lib/TestChainMapReduce.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/lib/TestChainMapReduce.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/lib/TestChainMapReduce.java
index 37cb91f..0933ece 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/lib/TestChainMapReduce.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/lib/TestChainMapReduce.java
@@ -21,12 +21,29 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.LongWritable;
 import org.apache.hadoop.io.Text;
-import org.apache.hadoop.mapred.*;
+import org.apache.hadoop.mapred.FileInputFormat;
+import org.apache.hadoop.mapred.FileOutputFormat;
+import org.apache.hadoop.mapred.HadoopTestCase;
+import org.apache.hadoop.mapred.JobClient;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.Mapper;
+import org.apache.hadoop.mapred.OutputCollector;
+import org.apache.hadoop.mapred.Reducer;
+import org.apache.hadoop.mapred.Reporter;
+import org.apache.hadoop.mapred.RunningJob;
+import org.apache.hadoop.mapred.TextInputFormat;
+import org.apache.hadoop.mapred.TextOutputFormat;
+import org.junit.Test;
 
 import java.io.DataOutputStream;
 import java.io.IOException;
 import java.util.Iterator;
 
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotSame;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
 public class TestChainMapReduce extends HadoopTestCase {
 
   private static Path getFlagDir(boolean local) {
@@ -67,6 +84,7 @@ public class TestChainMapReduce extends HadoopTestCase {
     super(HadoopTestCase.LOCAL_MR, HadoopTestCase.LOCAL_FS, 1, 1);
   }
 
+  @Test
   public void testChain() throws Exception {
     Path inDir = new Path("testing/chain/input");
     Path outDir = new Path("testing/chain/output");

http://git-wip-us.apache.org/repos/asf/hadoop/blob/247a7906/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/lib/TestKeyFieldBasedComparator.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/lib/TestKeyFieldBasedComparator.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/lib/TestKeyFieldBasedComparator.java
index 34a4d2c..35b3f24 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/lib/TestKeyFieldBasedComparator.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/lib/TestKeyFieldBasedComparator.java
@@ -36,6 +36,8 @@ import org.apache.hadoop.mapred.TextOutputFormat;
 import org.apache.hadoop.mapred.Utils;
 import org.junit.After;
 import org.junit.Test;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
 
 import java.io.BufferedReader;
 import java.io.File;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/247a7906/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/lib/TestMultipleInputs.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/lib/TestMultipleInputs.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/lib/TestMultipleInputs.java
index e5c6d75..3a9cb9e 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/lib/TestMultipleInputs.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/lib/TestMultipleInputs.java
@@ -17,11 +17,6 @@
  */
 package org.apache.hadoop.mapred.lib;
 
-import java.io.IOException;
-import java.util.Map;
-
-import junit.framework.TestCase;
-
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.mapred.InputFormat;
 import org.apache.hadoop.mapred.JobConf;
@@ -30,12 +25,19 @@ import org.apache.hadoop.mapred.Mapper;
 import org.apache.hadoop.mapred.OutputCollector;
 import org.apache.hadoop.mapred.Reporter;
 import org.apache.hadoop.mapred.TextInputFormat;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.util.Map;
+
+import static org.junit.Assert.assertEquals;
 
 /**
  * @see TestDelegatingInputFormat
  */
-public class TestMultipleInputs extends TestCase {
-  
+public class TestMultipleInputs {
+
+  @Test
   public void testAddInputPathWithFormat() {
     final JobConf conf = new JobConf();
     MultipleInputs.addInputPath(conf, new Path("/foo"), TextInputFormat.class);
@@ -48,6 +50,7 @@ public class TestMultipleInputs extends TestCase {
        .getClass());
   }
 
+  @Test
   public void testAddInputPathWithMapper() {
     final JobConf conf = new JobConf();
     MultipleInputs.addInputPath(conf, new Path("/foo"), TextInputFormat.class,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/247a7906/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/lib/TestMultipleOutputs.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/lib/TestMultipleOutputs.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/lib/TestMultipleOutputs.java
index 59c0a97..f3e5893 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/lib/TestMultipleOutputs.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/lib/TestMultipleOutputs.java
@@ -24,7 +24,23 @@ import org.apache.hadoop.io.LongWritable;
 import org.apache.hadoop.io.SequenceFile;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.serializer.JavaSerializationComparator;
-import org.apache.hadoop.mapred.*;
+import org.apache.hadoop.mapred.Counters;
+import org.apache.hadoop.mapred.FileInputFormat;
+import org.apache.hadoop.mapred.FileOutputFormat;
+import org.apache.hadoop.mapred.HadoopTestCase;
+import org.apache.hadoop.mapred.JobClient;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.Mapper;
+import org.apache.hadoop.mapred.OutputCollector;
+import org.apache.hadoop.mapred.Reducer;
+import org.apache.hadoop.mapred.Reporter;
+import org.apache.hadoop.mapred.RunningJob;
+import org.apache.hadoop.mapred.SequenceFileOutputFormat;
+import org.apache.hadoop.mapred.TextInputFormat;
+import org.apache.hadoop.mapred.TextOutputFormat;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
 
 import java.io.BufferedReader;
 import java.io.DataOutputStream;
@@ -32,17 +48,23 @@ import java.io.IOException;
 import java.io.InputStreamReader;
 import java.util.Iterator;
 
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
 public class TestMultipleOutputs extends HadoopTestCase {
 
   public TestMultipleOutputs() throws IOException {
     super(HadoopTestCase.LOCAL_MR, HadoopTestCase.LOCAL_FS, 1, 1);
   }
 
+  @Test
   public void testWithoutCounters() throws Exception {
     _testMultipleOutputs(false);
     _testMOWithJavaSerialization(false);
   }
 
+  @Test
   public void testWithCounters() throws Exception {
     _testMultipleOutputs(true);
     _testMOWithJavaSerialization(true);
@@ -62,6 +84,7 @@ public class TestMultipleOutputs extends HadoopTestCase {
     return dir;
   }
 
+  @Before
   public void setUp() throws Exception {
     super.setUp();
     Path rootDir = getDir(ROOT_DIR);
@@ -75,6 +98,7 @@ public class TestMultipleOutputs extends HadoopTestCase {
     }
   }
 
+  @After
   public void tearDown() throws Exception {
     Path rootDir = getDir(ROOT_DIR);
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/247a7906/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/lib/TestMultithreadedMapRunner.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/lib/TestMultithreadedMapRunner.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/lib/TestMultithreadedMapRunner.java
index 7e224cd..1059d29 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/lib/TestMultithreadedMapRunner.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/lib/TestMultithreadedMapRunner.java
@@ -22,26 +22,45 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.LongWritable;
 import org.apache.hadoop.io.Text;
-import org.apache.hadoop.mapred.*;
+import org.apache.hadoop.mapred.FileInputFormat;
+import org.apache.hadoop.mapred.FileOutputFormat;
+import org.apache.hadoop.mapred.HadoopTestCase;
+import org.apache.hadoop.mapred.JobClient;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.Mapper;
+import org.apache.hadoop.mapred.OutputCollector;
+import org.apache.hadoop.mapred.Reducer;
+import org.apache.hadoop.mapred.Reporter;
+import org.apache.hadoop.mapred.RunningJob;
+import org.apache.hadoop.mapred.TextInputFormat;
+import org.apache.hadoop.mapred.TextOutputFormat;
 import org.apache.hadoop.mapreduce.lib.map.MultithreadedMapper;
+import org.junit.Test;
 
 import java.io.DataOutputStream;
 import java.io.IOException;
 import java.util.Iterator;
 
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
 public class TestMultithreadedMapRunner extends HadoopTestCase {
 
   public TestMultithreadedMapRunner() throws IOException {
     super(HadoopTestCase.LOCAL_MR, HadoopTestCase.LOCAL_FS, 1, 1);
   }
 
+  @Test
   public void testOKRun() throws Exception {
     run(false, false);
   }
 
+  @Test
   public void testIOExRun() throws Exception {
     run(true, false);
   }
+
+  @Test
   public void testRuntimeExRun() throws Exception {
     run(false, true);
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/247a7906/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestChild.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestChild.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestChild.java
index d5afe63..338f117 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestChild.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestChild.java
@@ -30,6 +30,13 @@ import org.apache.hadoop.io.Text;
 import org.apache.hadoop.mapred.HadoopTestCase;
 import org.apache.hadoop.mapred.JobConf;
 import org.apache.log4j.Level;
+import org.junit.Before;
+import org.junit.Test;
+
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
 
 public class TestChild extends HadoopTestCase {
   private static String TEST_ROOT_DIR =
@@ -145,7 +152,8 @@ public class TestChild extends HadoopTestCase {
         + numPartFiles, list.length == numPartFiles);
     return job;
   }
-  
+
+  @Test
   public void testChild() throws Exception {
     try {
       submitAndValidateJob(createJobConf(), 1, 1, true);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/247a7906/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestNoJobSetupCleanup.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestNoJobSetupCleanup.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestNoJobSetupCleanup.java
index 5d36c92..7520f38 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestNoJobSetupCleanup.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestNoJobSetupCleanup.java
@@ -30,6 +30,9 @@ import org.apache.hadoop.mapred.HadoopTestCase;
 import org.apache.hadoop.mapreduce.lib.output.TextOutputFormat;
 import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl;
 import org.junit.Ignore;
+import org.junit.Test;
+import static org.junit.Assert.assertTrue;
+
 @Ignore
 public class TestNoJobSetupCleanup extends HadoopTestCase {
   private static String TEST_ROOT_DIR =
@@ -68,7 +71,8 @@ public class TestNoJobSetupCleanup extends HadoopTestCase {
         + numPartFiles, list.length == numPartFiles);
     return job;
   }
-  
+
+  @Test
   public void testNoJobSetupCleanup() throws Exception {
     try {
       Configuration conf = createJobConf();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/247a7906/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestTaskContext.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestTaskContext.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestTaskContext.java
index bf742c4..67daaa4 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestTaskContext.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestTaskContext.java
@@ -33,6 +33,8 @@ import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.Ignore;
 import org.junit.Test;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
 
 /**
  * Tests context api and {@link StatusReporter#getProgress()} via 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/247a7906/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/chain/TestChainErrors.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/chain/TestChainErrors.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/chain/TestChainErrors.java
index 2dfcf41..46024bc 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/chain/TestChainErrors.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/chain/TestChainErrors.java
@@ -29,6 +29,10 @@ import org.apache.hadoop.mapreduce.Job;
 import org.apache.hadoop.mapreduce.MapReduceTestUtil;
 import org.apache.hadoop.mapreduce.Mapper;
 import org.apache.hadoop.mapreduce.Reducer;
+import org.junit.Test;
+
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.assertEquals;
 
 /**
  * Tests error conditions in ChainMapper/ChainReducer.
@@ -51,6 +55,7 @@ public class TestChainErrors extends HadoopTestCase {
    * 
    * @throws Exception
    */
+  @Test
   public void testChainSubmission() throws Exception {
 
     Configuration conf = createJobConf();
@@ -89,6 +94,7 @@ public class TestChainErrors extends HadoopTestCase {
    * 
    * @throws Exception
    */
+  @Test
   public void testChainFail() throws Exception {
 
     Configuration conf = createJobConf();
@@ -114,6 +120,7 @@ public class TestChainErrors extends HadoopTestCase {
    * 
    * @throws Exception
    */
+  @Test
   public void testReducerFail() throws Exception {
 
     Configuration conf = createJobConf();
@@ -139,6 +146,7 @@ public class TestChainErrors extends HadoopTestCase {
    * 
    * @throws Exception
    */
+  @Test
   public void testChainMapNoOuptut() throws Exception {
     Configuration conf = createJobConf();
     String expectedOutput = "";
@@ -163,6 +171,7 @@ public class TestChainErrors extends HadoopTestCase {
    * 
    * @throws Exception
    */
+  @Test
   public void testChainReduceNoOuptut() throws Exception {
     Configuration conf = createJobConf();
     String expectedOutput = "";

http://git-wip-us.apache.org/repos/asf/hadoop/blob/247a7906/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/chain/TestMapReduceChain.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/chain/TestMapReduceChain.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/chain/TestMapReduceChain.java
index 971ea68..aaaaf51 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/chain/TestMapReduceChain.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/chain/TestMapReduceChain.java
@@ -30,6 +30,11 @@ import org.apache.hadoop.mapreduce.Job;
 import org.apache.hadoop.mapreduce.MapReduceTestUtil;
 import org.apache.hadoop.mapreduce.Mapper;
 import org.apache.hadoop.mapreduce.Reducer;
+import org.junit.Test;
+
+import static org.junit.Assert.fail;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
 
 public class TestMapReduceChain extends HadoopTestCase {
 
@@ -63,6 +68,7 @@ public class TestMapReduceChain extends HadoopTestCase {
     super(HadoopTestCase.LOCAL_MR, HadoopTestCase.LOCAL_FS, 1, 1);
   }
 
+  @Test
   public void testChain() throws Exception {
     Path inDir = new Path(localPathRoot, "testing/chain/input");
     Path outDir = new Path(localPathRoot, "testing/chain/output");

http://git-wip-us.apache.org/repos/asf/hadoop/blob/247a7906/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/chain/TestSingleElementChain.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/chain/TestSingleElementChain.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/chain/TestSingleElementChain.java
index 06cfe1c..f78ac70 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/chain/TestSingleElementChain.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/chain/TestSingleElementChain.java
@@ -26,6 +26,9 @@ import org.apache.hadoop.mapreduce.Job;
 import org.apache.hadoop.mapreduce.MapReduceTestUtil;
 import org.apache.hadoop.mapreduce.lib.map.TokenCounterMapper;
 import org.apache.hadoop.mapreduce.lib.reduce.IntSumReducer;
+import org.junit.Test;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
 
 import java.io.IOException;
 
@@ -42,6 +45,7 @@ public class TestSingleElementChain extends HadoopTestCase {
   }
 
   // test chain mapper and reducer by adding single mapper and reducer to chain
+  @Test
   public void testNoChain() throws Exception {
     Path inDir = new Path(localPathRoot, "testing/chain/input");
     Path outDir = new Path(localPathRoot, "testing/chain/output");

http://git-wip-us.apache.org/repos/asf/hadoop/blob/247a7906/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/db/TestDataDrivenDBInputFormat.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/db/TestDataDrivenDBInputFormat.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/db/TestDataDrivenDBInputFormat.java
index 37f9364..81a3249 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/db/TestDataDrivenDBInputFormat.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/db/TestDataDrivenDBInputFormat.java
@@ -18,25 +18,40 @@
 
 package org.apache.hadoop.mapreduce.lib.db;
 
-import java.sql.*;
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
-//import org.apache.hadoop.examples.DBCountPageView;
-import org.apache.hadoop.fs.*;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.NullWritable;
 import org.apache.hadoop.io.WritableComparable;
 import org.apache.hadoop.mapred.HadoopTestCase;
-import org.apache.hadoop.mapreduce.*;
-import org.apache.hadoop.mapreduce.lib.db.*;
-import org.apache.hadoop.mapreduce.lib.input.*;
-import org.apache.hadoop.mapreduce.lib.output.*;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.Mapper;
+import org.apache.hadoop.mapreduce.Reducer;
+import org.apache.hadoop.mapreduce.TaskCounter;
+import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
 import org.apache.hadoop.util.StringUtils;
 import org.hsqldb.server.Server;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.sql.Connection;
+import java.sql.Date;
+import java.sql.DriverManager;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.sql.Statement;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+//import org.apache.hadoop.examples.DBCountPageView;
 
 /**
  * Test aspects of DataDrivenDBInputFormat
@@ -109,11 +124,13 @@ public class TestDataDrivenDBInputFormat extends HadoopTestCase {
     createConnection(driverClassName, url);
   }
 
+  @Before
   public void setUp() throws Exception {
     initialize(DRIVER_CLASS, DB_URL);
     super.setUp();
   }
 
+  @After
   public void tearDown() throws Exception {
     super.tearDown();
     shutdown();
@@ -170,6 +187,7 @@ public class TestDataDrivenDBInputFormat extends HadoopTestCase {
     }
   }
 
+  @Test
   public void testDateSplits() throws Exception {
     Statement s = connection.createStatement();
     final String DATE_TABLE = "datetable";

http://git-wip-us.apache.org/repos/asf/hadoop/blob/247a7906/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/input/TestMultipleInputs.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/input/TestMultipleInputs.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/input/TestMultipleInputs.java
index c868050..632c40e 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/input/TestMultipleInputs.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/input/TestMultipleInputs.java
@@ -38,6 +38,8 @@ import org.apache.hadoop.mapreduce.Mapper;
 import org.apache.hadoop.mapreduce.Reducer;
 import org.junit.Before;
 import org.junit.Test;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
 
 /**
  * @see TestDelegatingInputFormat
@@ -139,7 +141,7 @@ public class TestMultipleInputs extends HadoopTestCase {
     assertTrue(output.readLine().equals("e 2"));
   }
 
-  @SuppressWarnings("unchecked")
+  @Test
   public void testAddInputPathWithFormat() throws IOException {
     final Job conf = Job.getInstance();
     MultipleInputs.addInputPath(conf, new Path("/foo"), TextInputFormat.class);
@@ -152,7 +154,7 @@ public class TestMultipleInputs extends HadoopTestCase {
        .getClass());
   }
 
-  @SuppressWarnings("unchecked")
+  @Test
   public void testAddInputPathWithMapper() throws IOException {
     final Job conf = Job.getInstance();
     MultipleInputs.addInputPath(conf, new Path("/foo"), TextInputFormat.class,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/247a7906/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/jobcontrol/TestMapReduceJobControl.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/jobcontrol/TestMapReduceJobControl.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/jobcontrol/TestMapReduceJobControl.java
index 14c64bd..d86ddd0 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/jobcontrol/TestMapReduceJobControl.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/jobcontrol/TestMapReduceJobControl.java
@@ -33,6 +33,9 @@ import org.apache.hadoop.mapred.HadoopTestCase;
 import org.apache.hadoop.mapreduce.Job;
 import org.apache.hadoop.mapreduce.MapReduceTestUtil;
 import org.junit.Test;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
 
 /**
  * This class performs unit test for Job/JobControl classes.
@@ -120,7 +123,8 @@ public class TestMapReduceJobControl extends HadoopTestCase {
       } catch (Exception e) {}
     }
   }
-  
+
+  @Test
   public void testJobControlWithFailJob() throws Exception {
     LOG.info("Starting testJobControlWithFailJob");
     Configuration conf = createJobConf();
@@ -144,6 +148,7 @@ public class TestMapReduceJobControl extends HadoopTestCase {
     theControl.stop();
   }
 
+  @Test
   public void testJobControlWithKillJob() throws Exception {
     LOG.info("Starting testJobControlWithKillJob");
 
@@ -182,6 +187,7 @@ public class TestMapReduceJobControl extends HadoopTestCase {
     theControl.stop();
   }
 
+  @Test
   public void testJobControl() throws Exception {
     LOG.info("Starting testJobControl");
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/247a7906/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/map/TestMultithreadedMapper.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/map/TestMultithreadedMapper.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/map/TestMultithreadedMapper.java
index 52b0e70..5096192 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/map/TestMultithreadedMapper.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/map/TestMultithreadedMapper.java
@@ -23,23 +23,33 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.LongWritable;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.mapred.HadoopTestCase;
-import org.apache.hadoop.mapreduce.*;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.MapReduceTestUtil;
+import org.apache.hadoop.mapreduce.Mapper;
+import org.apache.hadoop.mapreduce.Reducer;
+import org.junit.Test;
 
 import java.io.IOException;
 
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
 public class TestMultithreadedMapper extends HadoopTestCase {
 
   public TestMultithreadedMapper() throws IOException {
     super(HadoopTestCase.LOCAL_MR, HadoopTestCase.LOCAL_FS, 1, 1);
   }
 
+  @Test
   public void testOKRun() throws Exception {
     run(false, false);
   }
 
+  @Test
   public void testIOExRun() throws Exception {
     run(true, false);
   }
+  @Test
   public void testRuntimeExRun() throws Exception {
     run(false, true);
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/247a7906/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/output/TestJobOutputCommitter.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/output/TestJobOutputCommitter.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/output/TestJobOutputCommitter.java
index 49b59ca..19b712f 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/output/TestJobOutputCommitter.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/output/TestJobOutputCommitter.java
@@ -33,6 +33,11 @@ import org.apache.hadoop.mapreduce.MapReduceTestUtil;
 import org.apache.hadoop.mapreduce.OutputCommitter;
 import org.apache.hadoop.mapreduce.OutputFormat;
 import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.assertFalse;
 
 /**
  * A JUnit test to test Map-Reduce job committer.
@@ -54,15 +59,15 @@ public class TestJobOutputCommitter extends HadoopTestCase {
   private FileSystem fs;
   private Configuration conf = null;
 
-  @Override
-  protected void setUp() throws Exception {
+  @Before
+  public void setUp() throws Exception {
     super.setUp();
     conf = createJobConf();
     fs = getFileSystem();
   }
 
-  @Override
-  protected void tearDown() throws Exception {
+  @After
+  public void tearDown() throws Exception {
     fs.delete(new Path(TEST_ROOT_DIR), true);
     super.tearDown();
   }
@@ -219,6 +224,7 @@ public class TestJobOutputCommitter extends HadoopTestCase {
    * 
    * @throws Exception
    */
+  @Test
   public void testDefaultCleanupAndAbort() throws Exception {
     // check with a successful job
     testSuccessfulJob(FileOutputCommitter.SUCCEEDED_FILE_NAME,
@@ -238,6 +244,7 @@ public class TestJobOutputCommitter extends HadoopTestCase {
    * 
    * @throws Exception
    */
+  @Test
   public void testCustomAbort() throws Exception {
     // check with a successful job
     testSuccessfulJob(FileOutputCommitter.SUCCEEDED_FILE_NAME,
@@ -264,6 +271,7 @@ public class TestJobOutputCommitter extends HadoopTestCase {
    * compatibility testing.
    * @throws Exception 
    */
+  @Test
   public void testCustomCleanup() throws Exception {
     // check with a successful job
     testSuccessfulJob(CUSTOM_CLEANUP_FILE_NAME, 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/247a7906/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/output/TestMRMultipleOutputs.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/output/TestMRMultipleOutputs.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/output/TestMRMultipleOutputs.java
index 6c432dd..babd20e 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/output/TestMRMultipleOutputs.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/output/TestMRMultipleOutputs.java
@@ -27,23 +27,36 @@ import org.apache.hadoop.io.SequenceFile;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.serializer.JavaSerializationComparator;
 import org.apache.hadoop.mapred.HadoopTestCase;
-import org.apache.hadoop.mapreduce.*;
+import org.apache.hadoop.mapreduce.CounterGroup;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.MapReduceTestUtil;
+import org.apache.hadoop.mapreduce.Mapper;
+import org.apache.hadoop.mapreduce.Reducer;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
 
 import java.io.BufferedReader;
 import java.io.IOException;
 import java.io.InputStreamReader;
 
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
 public class TestMRMultipleOutputs extends HadoopTestCase {
 
   public TestMRMultipleOutputs() throws IOException {
     super(HadoopTestCase.LOCAL_MR, HadoopTestCase.LOCAL_FS, 1, 1);
   }
 
+  @Test
   public void testWithoutCounters() throws Exception {
     _testMultipleOutputs(false);
     _testMOWithJavaSerialization(false);
   }
 
+  @Test
   public void testWithCounters() throws Exception {
     _testMultipleOutputs(true);
     _testMOWithJavaSerialization(true);
@@ -57,6 +70,7 @@ public class TestMRMultipleOutputs extends HadoopTestCase {
   private static String TEXT = "text";
   private static String SEQUENCE = "sequence";
 
+  @Before
   public void setUp() throws Exception {
     super.setUp();
     Configuration conf = createJobConf();
@@ -64,6 +78,7 @@ public class TestMRMultipleOutputs extends HadoopTestCase {
     fs.delete(ROOT_DIR, true);
   }
 
+  @After
   public void tearDown() throws Exception {
     Configuration conf = createJobConf();
     FileSystem fs = FileSystem.get(conf);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/247a7906/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/partition/TestMRKeyFieldBasedComparator.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/partition/TestMRKeyFieldBasedComparator.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/partition/TestMRKeyFieldBasedComparator.java
index 3a2b831..0d75d2f 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/partition/TestMRKeyFieldBasedComparator.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/partition/TestMRKeyFieldBasedComparator.java
@@ -18,8 +18,6 @@
 
 package org.apache.hadoop.mapreduce.lib.partition;
 
-import java.io.*;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.Path;
@@ -32,6 +30,15 @@ import org.apache.hadoop.mapreduce.MRJobConfig;
 import org.apache.hadoop.mapreduce.MapReduceTestUtil;
 import org.apache.hadoop.mapreduce.Reducer;
 import org.apache.hadoop.mapreduce.lib.map.InverseMapper;
+import org.junit.Test;
+
+import java.io.BufferedReader;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
 
 
 public class TestMRKeyFieldBasedComparator extends HadoopTestCase {
@@ -45,8 +52,8 @@ public class TestMRKeyFieldBasedComparator extends HadoopTestCase {
     conf = createJobConf();
     conf.set(MRJobConfig.MAP_OUTPUT_KEY_FIELD_SEPERATOR, " ");
   }
-  
-  private void testComparator(String keySpec, int expect) 
+
+  private void testComparator(String keySpec, int expect)
       throws Exception {
     String root = System.getProperty("test.build.data", "/tmp");
     Path inDir = new Path(root, "test_cmp/in");
@@ -93,7 +100,8 @@ public class TestMRKeyFieldBasedComparator extends HadoopTestCase {
       reader.close();
     }
   }
-  
+
+  @Test
   public void testBasicUnixComparator() throws Exception {
     testComparator("-k1,1n", 1);
     testComparator("-k2,2n", 1);
@@ -117,7 +125,7 @@ public class TestMRKeyFieldBasedComparator extends HadoopTestCase {
   byte[] line1_bytes = line1.getBytes();
   byte[] line2_bytes = line2.getBytes();
 
-  public void testWithoutMRJob(String keySpec, int expect) throws Exception {
+  private void testWithoutMRJob(String keySpec, int expect) throws Exception {
     KeyFieldBasedComparator<Void, Void> keyFieldCmp = 
       new KeyFieldBasedComparator<Void, Void>();
     conf.set("mapreduce.partition.keycomparator.options", keySpec);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/247a7906/hadoop-mapreduce-project/hadoop-mapreduce-examples/src/test/java/org/apache/hadoop/examples/terasort/TestTeraSort.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-examples/src/test/java/org/apache/hadoop/examples/terasort/TestTeraSort.java b/hadoop-mapreduce-project/hadoop-mapreduce-examples/src/test/java/org/apache/hadoop/examples/terasort/TestTeraSort.java
index 3492089..391e482 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-examples/src/test/java/org/apache/hadoop/examples/terasort/TestTeraSort.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-examples/src/test/java/org/apache/hadoop/examples/terasort/TestTeraSort.java
@@ -27,6 +27,12 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.mapred.FileAlreadyExistsException;
 import org.apache.hadoop.mapred.HadoopTestCase;
 import org.apache.hadoop.util.ToolRunner;
+import org.junit.After;
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.fail;
+
 public class TestTeraSort extends HadoopTestCase {
   private static Log LOG = LogFactory.getLog(TestTeraSort.class);
   
@@ -35,7 +41,8 @@ public class TestTeraSort extends HadoopTestCase {
     super(LOCAL_MR, LOCAL_FS, 1, 1);
   }
 
-  protected void tearDown() throws Exception {
+  @After
+  public void tearDown() throws Exception {
     getFileSystem().delete(new Path(TEST_DIR), true);
     super.tearDown();
   }
@@ -76,6 +83,7 @@ public class TestTeraSort extends HadoopTestCase {
     assertEquals(ToolRunner.run(job, new TeraValidate(), svArgs), 0);
   }
 
+  @Test
   public void testTeraSort() throws Exception {
     // Run TeraGen to generate input for 'terasort'
     runTeraGen(createJobConf(), SORT_INPUT_PATH);
@@ -104,6 +112,7 @@ public class TestTeraSort extends HadoopTestCase {
       TERA_OUTPUT_PATH);
   }
 
+  @Test
   public void testTeraSortWithLessThanTwoArgs() throws Exception {
     String[] args = new String[1];
     assertEquals(new TeraSort().run(args), 2);


[05/46] hadoop git commit: Revert "HADOOP-12672. RPC timeout should not override IPC ping interval (iwasakims)"

Posted by ae...@apache.org.
Revert "HADOOP-12672. RPC timeout should not override IPC ping interval (iwasakims)"

This reverts commit 682adc6ba9db3bed94fd4ea3d83761db6abfe695.


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

Branch: refs/heads/HDFS-7240
Commit: 754299695b778b9b602e46836c35a3ac9474d7f8
Parents: 247a790
Author: Steve Loughran <st...@apache.org>
Authored: Fri Mar 11 17:00:17 2016 +0000
Committer: Steve Loughran <st...@apache.org>
Committed: Fri Mar 11 17:00:17 2016 +0000

----------------------------------------------------------------------
 .../main/java/org/apache/hadoop/ipc/Client.java | 33 ++++------
 .../src/main/resources/core-default.xml         |  9 +--
 .../java/org/apache/hadoop/ipc/TestRPC.java     | 68 --------------------
 3 files changed, 19 insertions(+), 91 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/75429969/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Client.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Client.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Client.java
index 3ae1d67..8d87957 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Client.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Client.java
@@ -386,7 +386,7 @@ public class Client {
     private Socket socket = null;                 // connected socket
     private DataInputStream in;
     private DataOutputStream out;
-    private final int rpcTimeout;
+    private int rpcTimeout;
     private int maxIdleTime; //connections will be culled if it was idle for 
     //maxIdleTime msecs
     private final RetryPolicy connectionRetryPolicy;
@@ -394,9 +394,8 @@ public class Client {
     private int maxRetriesOnSocketTimeouts;
     private final boolean tcpNoDelay; // if T then disable Nagle's Algorithm
     private final boolean tcpLowLatency; // if T then use low-delay QoS
-    private final boolean doPing; //do we need to send ping message
-    private final int pingInterval; // how often sends ping to the server
-    private final int soTimeout; // used by ipc ping and rpc timeout
+    private boolean doPing; //do we need to send ping message
+    private int pingInterval; // how often sends ping to the server in msecs
     private ByteArrayOutputStream pingRequest; // ping message
     
     // currently active calls
@@ -435,9 +434,6 @@ public class Client {
         pingHeader.writeDelimitedTo(pingRequest);
       }
       this.pingInterval = remoteId.getPingInterval();
-      this.soTimeout =
-          (rpcTimeout == 0 || (doPing && pingInterval < rpcTimeout))?
-              this.pingInterval : this.rpcTimeout;
       this.serviceClass = serviceClass;
       if (LOG.isDebugEnabled()) {
         LOG.debug("The ping interval is " + this.pingInterval + " ms.");
@@ -488,12 +484,12 @@ public class Client {
 
       /* Process timeout exception
        * if the connection is not going to be closed or 
-       * the RPC is not timed out yet, send a ping.
+       * is not configured to have a RPC timeout, send a ping.
+       * (if rpcTimeout is not set to be 0, then RPC should timeout.
+       * otherwise, throw the timeout exception.
        */
-      private void handleTimeout(SocketTimeoutException e, int waiting)
-          throws IOException {
-        if (shouldCloseConnection.get() || !running.get() ||
-            (0 < rpcTimeout && rpcTimeout <= waiting)) {
+      private void handleTimeout(SocketTimeoutException e) throws IOException {
+        if (shouldCloseConnection.get() || !running.get() || rpcTimeout > 0) {
           throw e;
         } else {
           sendPing();
@@ -507,13 +503,11 @@ public class Client {
        */
       @Override
       public int read() throws IOException {
-        int waiting = 0;
         do {
           try {
             return super.read();
           } catch (SocketTimeoutException e) {
-            waiting += soTimeout;
-            handleTimeout(e, waiting);
+            handleTimeout(e);
           }
         } while (true);
       }
@@ -526,13 +520,11 @@ public class Client {
        */
       @Override
       public int read(byte[] buf, int off, int len) throws IOException {
-        int waiting = 0;
         do {
           try {
             return super.read(buf, off, len);
           } catch (SocketTimeoutException e) {
-            waiting += soTimeout;
-            handleTimeout(e, waiting);
+            handleTimeout(e);
           }
         } while (true);
       }
@@ -640,7 +632,10 @@ public class Client {
           }
           
           NetUtils.connect(this.socket, server, connectionTimeout);
-          this.socket.setSoTimeout(soTimeout);
+          if (rpcTimeout > 0) {
+            pingInterval = rpcTimeout;  // rpcTimeout overwrites pingInterval
+          }
+          this.socket.setSoTimeout(pingInterval);
           return;
         } catch (ConnectTimeoutException toe) {
           /* Check for an address change and update the local reference.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/75429969/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml b/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml
index 5037113..187f923 100644
--- a/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml
+++ b/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml
@@ -1054,7 +1054,7 @@
   <value>true</value>
   <description>Send a ping to the server when timeout on reading the response,
   if set to true. If no failure is detected, the client retries until at least
-  a byte is read or the time given by ipc.client.rpc-timeout.ms is passed.
+  a byte is read.
   </description>
 </property>
 
@@ -1071,9 +1071,10 @@
   <name>ipc.client.rpc-timeout.ms</name>
   <value>0</value>
   <description>Timeout on waiting response from server, in milliseconds.
-  If ipc.client.ping is set to true and this rpc-timeout is greater than
-  the value of ipc.ping.interval, the effective value of the rpc-timeout is
-  rounded up to multiple of ipc.ping.interval.
+  Currently this timeout works only when ipc.client.ping is set to true
+  because it uses the same facilities with IPC ping.
+  The timeout overrides the ipc.ping.interval and client will throw exception
+  instead of sending ping when the interval is passed.
   </description>
 </property>
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/75429969/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestRPC.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestRPC.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestRPC.java
index 929b82b..99bfc61 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestRPC.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestRPC.java
@@ -1043,74 +1043,6 @@ public class TestRPC extends TestRpcBase {
     }
   }
 
-  /**
-   *  Test RPC timeout when ipc.client.ping is false.
-   */
-  @Test(timeout=30000)
-  public void testClientRpcTimeoutWithoutPing() throws Exception {
-    final Server server = new RPC.Builder(conf)
-        .setProtocol(TestProtocol.class).setInstance(new TestImpl())
-        .setBindAddress(ADDRESS).setPort(0)
-        .setQueueSizePerHandler(1).setNumHandlers(1).setVerbose(true)
-        .build();
-    server.start();
-
-    final Configuration conf = new Configuration();
-    conf.setBoolean(CommonConfigurationKeys.IPC_CLIENT_PING_KEY, false);
-    conf.setInt(CommonConfigurationKeys.IPC_CLIENT_RPC_TIMEOUT_KEY, 1000);
-    final TestProtocol proxy =
-        RPC.getProxy(TestProtocol.class, TestProtocol.versionID,
-            NetUtils.getConnectAddress(server), conf);
-
-    try {
-      proxy.sleep(3000);
-      fail("RPC should time out.");
-    } catch (SocketTimeoutException e) {
-      LOG.info("got expected timeout.", e);
-    } finally {
-      server.stop();
-      RPC.stopProxy(proxy);
-    }
-  }
-
-  /**
-   *  Test RPC timeout greater than ipc.ping.interval.
-   */
-  @Test(timeout=30000)
-  public void testClientRpcTimeoutGreaterThanPingInterval() throws Exception {
-    final Server server = new RPC.Builder(conf)
-        .setProtocol(TestProtocol.class).setInstance(new TestImpl())
-        .setBindAddress(ADDRESS).setPort(0)
-        .setQueueSizePerHandler(1).setNumHandlers(1).setVerbose(true)
-        .build();
-    server.start();
-
-    final Configuration conf = new Configuration();
-    conf.setBoolean(CommonConfigurationKeys.IPC_CLIENT_PING_KEY, true);
-    conf.setInt(CommonConfigurationKeys.IPC_PING_INTERVAL_KEY, 800);
-    conf.setInt(CommonConfigurationKeys.IPC_CLIENT_RPC_TIMEOUT_KEY, 1000);
-    final TestProtocol proxy =
-        RPC.getProxy(TestProtocol.class, TestProtocol.versionID,
-            NetUtils.getConnectAddress(server), conf);
-
-    // should not time out.
-    proxy.sleep(300);
-
-    // should not time out because effective rpc-timeout is
-    // multiple of ping interval: 1600 (= 800 * (1000 / 800 + 1))
-    proxy.sleep(1300);
-
-    try {
-      proxy.sleep(2000);
-      fail("RPC should time out.");
-    } catch (SocketTimeoutException e) {
-      LOG.info("got expected timeout.", e);
-    } finally {
-      server.stop();
-      RPC.stopProxy(proxy);
-    }
-  }
-
   public static void main(String[] args) throws Exception {
     new TestRPC().testCallsInternal(conf);
   }


[22/46] hadoop git commit: MAPREDUCE-6579. JobStatus#getFailureInfo should not output diagnostic information when the job is running. (Akira AJISAKA via wangda)

Posted by ae...@apache.org.
MAPREDUCE-6579. JobStatus#getFailureInfo should not output diagnostic information when the job is running. (Akira AJISAKA via wangda)


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

Branch: refs/heads/HDFS-7240
Commit: 6529c87551e07e9da1e1b273eff4c6bad6b3e838
Parents: 5176a69
Author: Wangda Tan <wa...@apache.org>
Authored: Tue Mar 15 17:28:40 2016 -0700
Committer: Wangda Tan <wa...@apache.org>
Committed: Tue Mar 15 17:28:40 2016 -0700

----------------------------------------------------------------------
 .../org/apache/hadoop/mapred/ClusterStatus.java | 35 +++++++-
 .../org/apache/hadoop/mapred/NotRunningJob.java |  7 +-
 .../apache/hadoop/mapred/TestNetworkedJob.java  | 89 ++++++++++----------
 3 files changed, 84 insertions(+), 47 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/6529c875/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/ClusterStatus.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/ClusterStatus.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/ClusterStatus.java
index 904897b..c87b583 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/ClusterStatus.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/ClusterStatus.java
@@ -171,7 +171,40 @@ public class ClusterStatus implements Writable {
       sb.append(blackListReport.replace("\n", ":"));
       return sb.toString();
     }
-    
+
+    @Override
+    public int hashCode() {
+      int result = trackerName != null ? trackerName.hashCode() : 0;
+      result = 31 * result + (reasonForBlackListing != null ?
+          reasonForBlackListing.hashCode() : 0);
+      result = 31 * result + (blackListReport != null ?
+          blackListReport.hashCode() : 0);
+      return result;
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+      if (this == obj) {
+        return true;
+      }
+      if (obj == null || getClass() != obj.getClass()) {
+        return false;
+      }
+      final BlackListInfo that = (BlackListInfo) obj;
+      if (trackerName == null ? that.trackerName != null :
+          !trackerName.equals(that.trackerName)) {
+        return false;
+      }
+      if (reasonForBlackListing == null ? that.reasonForBlackListing != null :
+          !reasonForBlackListing.equals(that.reasonForBlackListing)) {
+        return false;
+      }
+      if (blackListReport == null ? that.blackListReport != null :
+          !blackListReport.equals(that.blackListReport)) {
+        return false;
+      }
+      return true;
+    }
   }
   
   public static final long UNINITIALIZED_MEMORY_VALUE = -1;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6529c875/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/main/java/org/apache/hadoop/mapred/NotRunningJob.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/main/java/org/apache/hadoop/mapred/NotRunningJob.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/main/java/org/apache/hadoop/mapred/NotRunningJob.java
index 03552e4..90635a6 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/main/java/org/apache/hadoop/mapred/NotRunningJob.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/main/java/org/apache/hadoop/mapred/NotRunningJob.java
@@ -137,7 +137,12 @@ public class NotRunningJob implements MRClientProtocol {
     jobReport.setJobState(jobState);
     jobReport.setUser(applicationReport.getUser());
     jobReport.setStartTime(applicationReport.getStartTime());
-    jobReport.setDiagnostics(applicationReport.getDiagnostics());
+    YarnApplicationState state = applicationReport.getYarnApplicationState();
+    if (state == YarnApplicationState.KILLED
+        || state == YarnApplicationState.FAILED
+        || state == YarnApplicationState.FINISHED) {
+      jobReport.setDiagnostics(applicationReport.getDiagnostics());
+    }
     jobReport.setJobName(applicationReport.getName());
     jobReport.setTrackingUrl(applicationReport.getTrackingUrl());
     jobReport.setFinishTime(applicationReport.getFinishTime());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6529c875/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestNetworkedJob.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestNetworkedJob.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestNetworkedJob.java
index 2e0887e..45c7d1f 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestNetworkedJob.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestNetworkedJob.java
@@ -155,9 +155,9 @@ public class TestNetworkedJob {
       // test getters
       assertTrue(runningJob.getConfiguration().toString()
           .endsWith("0001/job.xml"));
-      assertEquals(runningJob.getID(), jobId);
-      assertEquals(runningJob.getJobID(), jobId.toString());
-      assertEquals(runningJob.getJobName(), "N/A");
+      assertEquals(jobId, runningJob.getID());
+      assertEquals(jobId.toString(), runningJob.getJobID());
+      assertEquals("N/A", runningJob.getJobName());
       assertTrue(runningJob.getJobFile().endsWith(
           ".staging/" + runningJob.getJobID() + "/job.xml"));
       assertTrue(runningJob.getTrackingURL().length() > 0);
@@ -169,40 +169,40 @@ public class TestNetworkedJob {
       TaskCompletionEvent[] tce = runningJob.getTaskCompletionEvents(0);
       assertEquals(tce.length, 0);
 
-      assertEquals(runningJob.getHistoryUrl(),"");
+      assertEquals("", runningJob.getHistoryUrl());
       assertFalse(runningJob.isRetired());
-      assertEquals( runningJob.getFailureInfo(),"");
-      assertEquals(runningJob.getJobStatus().getJobName(), "N/A");
-      assertEquals(client.getMapTaskReports(jobId).length, 0);
+      assertEquals("", runningJob.getFailureInfo());
+      assertEquals("N/A", runningJob.getJobStatus().getJobName());
+      assertEquals(0, client.getMapTaskReports(jobId).length);
       
       try {
         client.getSetupTaskReports(jobId);
       } catch (YarnRuntimeException e) {
-        assertEquals(e.getMessage(), "Unrecognized task type: JOB_SETUP");
+        assertEquals("Unrecognized task type: JOB_SETUP", e.getMessage());
       }
       try {
         client.getCleanupTaskReports(jobId);
       } catch (YarnRuntimeException e) {
-        assertEquals(e.getMessage(), "Unrecognized task type: JOB_CLEANUP");
+        assertEquals("Unrecognized task type: JOB_CLEANUP", e.getMessage());
       }
-      assertEquals(client.getReduceTaskReports(jobId).length, 0);
+      assertEquals(0, client.getReduceTaskReports(jobId).length);
       // test ClusterStatus
       ClusterStatus status = client.getClusterStatus(true);
-      assertEquals(status.getActiveTrackerNames().size(), 2);
+      assertEquals(2, status.getActiveTrackerNames().size());
       // it method does not implemented and always return empty array or null;
-      assertEquals(status.getBlacklistedTrackers(), 0);
-      assertEquals(status.getBlacklistedTrackerNames().size(), 0);
-      assertEquals(status.getBlackListedTrackersInfo().size(), 0);
-      assertEquals(status.getJobTrackerStatus(), JobTrackerStatus.RUNNING);
-      assertEquals(status.getMapTasks(), 1);
-      assertEquals(status.getMaxMapTasks(), 20);
-      assertEquals(status.getMaxReduceTasks(), 4);
-      assertEquals(status.getNumExcludedNodes(), 0);
-      assertEquals(status.getReduceTasks(), 1);
-      assertEquals(status.getTaskTrackers(), 2);
-      assertEquals(status.getTTExpiryInterval(), 0);
-      assertEquals(status.getJobTrackerStatus(), JobTrackerStatus.RUNNING);
-      assertEquals(status.getGraylistedTrackers(), 0);
+      assertEquals(0, status.getBlacklistedTrackers());
+      assertEquals(0, status.getBlacklistedTrackerNames().size());
+      assertEquals(0, status.getBlackListedTrackersInfo().size());
+      assertEquals(JobTrackerStatus.RUNNING, status.getJobTrackerStatus());
+      assertEquals(1, status.getMapTasks());
+      assertEquals(20, status.getMaxMapTasks());
+      assertEquals(4, status.getMaxReduceTasks());
+      assertEquals(0, status.getNumExcludedNodes());
+      assertEquals(1, status.getReduceTasks());
+      assertEquals(2, status.getTaskTrackers());
+      assertEquals(0, status.getTTExpiryInterval());
+      assertEquals(JobTrackerStatus.RUNNING, status.getJobTrackerStatus());
+      assertEquals(0, status.getGraylistedTrackers());
 
       // test read and write
       ByteArrayOutputStream dataOut = new ByteArrayOutputStream();
@@ -219,34 +219,34 @@ public class TestNetworkedJob {
 
       // test taskStatusfilter
       JobClient.setTaskOutputFilter(job, TaskStatusFilter.ALL);
-      assertEquals(JobClient.getTaskOutputFilter(job), TaskStatusFilter.ALL);
+      assertEquals(TaskStatusFilter.ALL, JobClient.getTaskOutputFilter(job));
 
       // runningJob.setJobPriority(JobPriority.HIGH.name());
 
       // test default map
-      assertEquals(client.getDefaultMaps(), 20);
-      assertEquals(client.getDefaultReduces(), 4);
-      assertEquals(client.getSystemDir().getName(), "jobSubmitDir");
+      assertEquals(20, client.getDefaultMaps());
+      assertEquals(4, client.getDefaultReduces());
+      assertEquals("jobSubmitDir", client.getSystemDir().getName());
       // test queue information
       JobQueueInfo[] rootQueueInfo = client.getRootQueues();
-      assertEquals(rootQueueInfo.length, 1);
-      assertEquals(rootQueueInfo[0].getQueueName(), "default");
+      assertEquals(1, rootQueueInfo.length);
+      assertEquals("default", rootQueueInfo[0].getQueueName());
       JobQueueInfo[] qinfo = client.getQueues();
-      assertEquals(qinfo.length, 1);
-      assertEquals(qinfo[0].getQueueName(), "default");
-      assertEquals(client.getChildQueues("default").length, 0);
-      assertEquals(client.getJobsFromQueue("default").length, 1);
+      assertEquals(1, qinfo.length);
+      assertEquals("default", qinfo[0].getQueueName());
+      assertEquals(0, client.getChildQueues("default").length);
+      assertEquals(1, client.getJobsFromQueue("default").length);
       assertTrue(client.getJobsFromQueue("default")[0].getJobFile().endsWith(
           "/job.xml"));
 
       JobQueueInfo qi = client.getQueueInfo("default");
-      assertEquals(qi.getQueueName(), "default");
-      assertEquals(qi.getQueueState(), "running");
+      assertEquals("default", qi.getQueueName());
+      assertEquals("running", qi.getQueueState());
 
       QueueAclsInfo[] aai = client.getQueueAclsForCurrentUser();
-      assertEquals(aai.length, 2);
-      assertEquals(aai[0].getQueueName(), "root");
-      assertEquals(aai[1].getQueueName(), "default");
+      assertEquals(2, aai.length);
+      assertEquals("root", aai[0].getQueueName());
+      assertEquals("default", aai[1].getQueueName());
       
       // test JobClient
       // The following asserts read JobStatus twice and ensure the returned
@@ -282,12 +282,11 @@ public class TestNetworkedJob {
     BlackListInfo info2 = new BlackListInfo();
     info2.readFields(new DataInputStream(new ByteArrayInputStream(byteOut
         .toByteArray())));
-    assertEquals(info, info);
-    assertEquals(info.toString(), info.toString());
-    assertEquals(info.getTrackerName(), "trackerName");
-    assertEquals(info.getReasonForBlackListing(), "reasonForBlackListing");
-    assertEquals(info.getBlackListReport(), "blackListInfo");
-
+    assertEquals(info, info2);
+    assertEquals(info.toString(), info2.toString());
+    assertEquals("trackerName", info2.getTrackerName());
+    assertEquals("reasonForBlackListing", info2.getReasonForBlackListing());
+    assertEquals("blackListInfo", info2.getBlackListReport());
   }
 /**
  *  test run from command line JobQueueClient


[25/46] hadoop git commit: HADOOP-12888 Shell to disable bash and setsid support when running under JVM security manager (Costin Leau via stevel)

Posted by ae...@apache.org.
HADOOP-12888 Shell to disable bash and setsid support when running under JVM security manager (Costin Leau via stevel)


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

Branch: refs/heads/HDFS-7240
Commit: 5a725f0ab8ef9e2a8b08f088ba4e87531ae4530d
Parents: 3ef5500
Author: Steve Loughran <st...@apache.org>
Authored: Wed Mar 16 14:31:19 2016 +0000
Committer: Steve Loughran <st...@apache.org>
Committed: Wed Mar 16 14:31:19 2016 +0000

----------------------------------------------------------------------
 .../src/main/java/org/apache/hadoop/util/Shell.java       | 10 +++++++++-
 1 file changed, 9 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/5a725f0a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/Shell.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/Shell.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/Shell.java
index df9ffa7..0af3752 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/Shell.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/Shell.java
@@ -717,6 +717,10 @@ public abstract class Shell {
     } catch (IOException ioe) {
       LOG.warn("Bash is not supported by the OS", ioe);
       supported = false;
+    } catch (SecurityException se) {
+      LOG.info("Bash execution is not allowed by the JVM " +
+          "security manager.Considering it not supported.");
+      supported = false;
     }
 
     return supported;
@@ -744,7 +748,11 @@ public abstract class Shell {
     } catch (IOException ioe) {
       LOG.debug("setsid is not available on this machine. So not using it.");
       setsidSupported = false;
-    }  catch (Error err) {
+    } catch (SecurityException se) {
+      LOG.debug("setsid is not allowed to run by the JVM "+
+          "security manager. So not using it.");
+      setsidSupported = false;
+    } catch (Error err) {
       if (err.getMessage() != null
           && err.getMessage().contains("posix_spawn is not " +
           "a supported process launch mechanism")


[20/46] hadoop git commit: YARN-4814. ATS 1.5 timelineclient impl call flush after every event write. (Xuan Gong via gtcarrera9)

Posted by ae...@apache.org.
YARN-4814. ATS 1.5 timelineclient impl call flush after every event write. (Xuan Gong via gtcarrera9)


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

Branch: refs/heads/HDFS-7240
Commit: 3a0b69544183f71c4683bc477656473494fc09d0
Parents: 22ca176
Author: Li Lu <gt...@apache.org>
Authored: Tue Mar 15 11:40:54 2016 -0700
Committer: Li Lu <gt...@apache.org>
Committed: Tue Mar 15 11:40:54 2016 -0700

----------------------------------------------------------------------
 .../hadoop/yarn/client/api/impl/FileSystemTimelineWriter.java       | 1 +
 1 file changed, 1 insertion(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/3a0b6954/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/impl/FileSystemTimelineWriter.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/impl/FileSystemTimelineWriter.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/impl/FileSystemTimelineWriter.java
index 9e719b7..44cad60 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/impl/FileSystemTimelineWriter.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/impl/FileSystemTimelineWriter.java
@@ -276,6 +276,7 @@ public class FileSystemTimelineWriter extends TimelineWriter{
     mapper.setAnnotationIntrospector(new JaxbAnnotationIntrospector());
     mapper.setSerializationInclusion(Inclusion.NON_NULL);
     mapper.configure(Feature.CLOSE_CLOSEABLE, false);
+    mapper.configure(Feature.FLUSH_AFTER_WRITE_VALUE, false);
     return mapper;
   }
 


[23/46] hadoop git commit: Revert "YARN-4814. ATS 1.5 timelineclient impl call flush after every event write. (Xuan Gong via gtcarrera9)"

Posted by ae...@apache.org.
Revert "YARN-4814. ATS 1.5 timelineclient impl call flush after every event write. (Xuan Gong via gtcarrera9)"

This reverts commit 3a0b69544183f71c4683bc477656473494fc09d0.


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

Branch: refs/heads/HDFS-7240
Commit: a888b5bc533bb91a3394f15a69879be1d7838592
Parents: 6529c875
Author: Xuan <xg...@apache.org>
Authored: Tue Mar 15 22:10:39 2016 -0700
Committer: Xuan <xg...@apache.org>
Committed: Tue Mar 15 22:10:39 2016 -0700

----------------------------------------------------------------------
 .../hadoop/yarn/client/api/impl/FileSystemTimelineWriter.java       | 1 -
 1 file changed, 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/a888b5bc/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/impl/FileSystemTimelineWriter.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/impl/FileSystemTimelineWriter.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/impl/FileSystemTimelineWriter.java
index c494f65..b471b3b 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/impl/FileSystemTimelineWriter.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/impl/FileSystemTimelineWriter.java
@@ -276,7 +276,6 @@ public class FileSystemTimelineWriter extends TimelineWriter{
     mapper.setAnnotationIntrospector(new JaxbAnnotationIntrospector());
     mapper.setSerializationInclusion(Inclusion.NON_NULL);
     mapper.configure(Feature.CLOSE_CLOSEABLE, false);
-    mapper.configure(Feature.FLUSH_AFTER_WRITE_VALUE, false);
     return mapper;
   }
 


[12/46] hadoop git commit: Download File from UI broken after pagination. Contributed by Brahma Reddy Battula

Posted by ae...@apache.org.
Download File from UI broken after pagination. 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/bd5556ba
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/bd5556ba
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/bd5556ba

Branch: refs/heads/HDFS-7240
Commit: bd5556ba124a5d9ec3e3f2041d96a2c445d69369
Parents: 20d389c
Author: Ravi Prakash <ra...@altiscale.com>
Authored: Mon Mar 14 14:30:13 2016 -0700
Committer: Ravi Prakash <ra...@altiscale.com>
Committed: Mon Mar 14 14:30:13 2016 -0700

----------------------------------------------------------------------
 .../src/main/webapps/hdfs/explorer.js           | 33 ++++++++++----------
 1 file changed, 17 insertions(+), 16 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/bd5556ba/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/explorer.js
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/explorer.js b/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/explorer.js
index baca798..102da9d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/explorer.js
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/explorer.js
@@ -281,21 +281,6 @@
       dust.render('explorer', base.push(d), function(err, out) {
         $('#panel').html(out);
 
-        $('#table-explorer').dataTable( {
-          'lengthMenu': [ [25, 50, 100, -1], [25, 50, 100, "All"] ],
-          'columns': [
-            {'searchable': false }, //Permissions
-            null, //Owner
-            null, //Group
-            { 'searchable': false, 'render': func_size_render}, //Size
-            { 'searchable': false, 'render': func_time_render}, //Last Modified
-            { 'searchable': false }, //Replication
-            null, //Block Size
-            null, //Name
-            { 'sortable' : false } //Trash
-          ],
-          "deferRender": true
-        });
 
         $('.explorer-browse-links').click(function() {
           var type = $(this).attr('inode-type');
@@ -324,7 +309,23 @@
           var inode_name = $(this).closest('tr').attr('inode-path');
           var absolute_file_path = append_path(current_directory, inode_name);
           delete_path(inode_name, absolute_file_path);
-        })
+        });
+          
+          $('#table-explorer').dataTable( {
+              'lengthMenu': [ [25, 50, 100, -1], [25, 50, 100, "All"] ],
+              'columns': [
+                  {'searchable': false }, //Permissions
+                  null, //Owner
+                  null, //Group
+                  { 'searchable': false, 'render': func_size_render}, //Size
+                  { 'searchable': false, 'render': func_time_render}, //Last Modified
+                  { 'searchable': false }, //Replication
+                  null, //Block Size
+                  null, //Name
+                  { 'sortable' : false } //Trash
+              ],
+              "deferRender": true
+          });
       });
     }).error(network_error_handler(url));
   }


[07/46] hadoop git commit: HADOOP-11996. Improve and restructure native ISAL support (Kai Zheng via cmccabe)

Posted by ae...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/658ee95f/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/erasurecode/org_apache_hadoop_io_erasurecode_rawcoder_NativeRSRawDecoder.h
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/erasurecode/org_apache_hadoop_io_erasurecode_rawcoder_NativeRSRawDecoder.h b/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/erasurecode/org_apache_hadoop_io_erasurecode_rawcoder_NativeRSRawDecoder.h
new file mode 100644
index 0000000..40da4e1
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/erasurecode/org_apache_hadoop_io_erasurecode_rawcoder_NativeRSRawDecoder.h
@@ -0,0 +1,37 @@
+/* DO NOT EDIT THIS FILE - it is machine generated */
+#include <jni.h>
+/* Header for class org_apache_hadoop_io_erasurecode_rawcoder_NativeRSRawDecoder */
+
+#ifndef _Included_org_apache_hadoop_io_erasurecode_rawcoder_NativeRSRawDecoder
+#define _Included_org_apache_hadoop_io_erasurecode_rawcoder_NativeRSRawDecoder
+#ifdef __cplusplus
+extern "C" {
+#endif
+/*
+ * Class:     org_apache_hadoop_io_erasurecode_rawcoder_NativeRSRawDecoder
+ * Method:    initImpl
+ * Signature: (II[I)V
+ */
+JNIEXPORT void JNICALL Java_org_apache_hadoop_io_erasurecode_rawcoder_NativeRSRawDecoder_initImpl
+  (JNIEnv *, jobject, jint, jint);
+
+/*
+ * Class:     org_apache_hadoop_io_erasurecode_rawcoder_NativeRSRawDecoder
+ * Method:    decodeImpl
+ * Signature: ([Ljava/nio/ByteBuffer;[II[I[Ljava/nio/ByteBuffer;[I)V
+ */
+JNIEXPORT void JNICALL Java_org_apache_hadoop_io_erasurecode_rawcoder_NativeRSRawDecoder_decodeImpl
+  (JNIEnv *, jobject, jobjectArray, jintArray, jint, jintArray, jobjectArray, jintArray);
+
+/*
+ * Class:     org_apache_hadoop_io_erasurecode_rawcoder_NativeRSRawDecoder
+ * Method:    destroyImpl
+ * Signature: ()V
+ */
+JNIEXPORT void JNICALL Java_org_apache_hadoop_io_erasurecode_rawcoder_NativeRSRawDecoder_destroyImpl
+  (JNIEnv *, jobject);
+
+#ifdef __cplusplus
+}
+#endif
+#endif

http://git-wip-us.apache.org/repos/asf/hadoop/blob/658ee95f/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/erasurecode/org_apache_hadoop_io_erasurecode_rawcoder_NativeRSRawEncoder.h
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/erasurecode/org_apache_hadoop_io_erasurecode_rawcoder_NativeRSRawEncoder.h b/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/erasurecode/org_apache_hadoop_io_erasurecode_rawcoder_NativeRSRawEncoder.h
new file mode 100644
index 0000000..db094cf
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/erasurecode/org_apache_hadoop_io_erasurecode_rawcoder_NativeRSRawEncoder.h
@@ -0,0 +1,37 @@
+/* DO NOT EDIT THIS FILE - it is machine generated */
+#include <jni.h>
+/* Header for class org_apache_hadoop_io_erasurecode_rawcoder_NativeRSRawEncoder */
+
+#ifndef _Included_org_apache_hadoop_io_erasurecode_rawcoder_NativeRSRawEncoder
+#define _Included_org_apache_hadoop_io_erasurecode_rawcoder_NativeRSRawEncoder
+#ifdef __cplusplus
+extern "C" {
+#endif
+/*
+ * Class:     org_apache_hadoop_io_erasurecode_rawcoder_NativeRSRawEncoder
+ * Method:    initImpl
+ * Signature: (II[I)V
+ */
+JNIEXPORT void JNICALL Java_org_apache_hadoop_io_erasurecode_rawcoder_NativeRSRawEncoder_initImpl
+  (JNIEnv *, jobject, jint, jint);
+
+/*
+ * Class:     org_apache_hadoop_io_erasurecode_rawcoder_NativeRSRawEncoder
+ * Method:    encodeImpl
+ * Signature: ([Ljava/nio/ByteBuffer;[II[Ljava/nio/ByteBuffer;[I)V
+ */
+JNIEXPORT void JNICALL Java_org_apache_hadoop_io_erasurecode_rawcoder_NativeRSRawEncoder_encodeImpl
+  (JNIEnv *, jobject, jobjectArray, jintArray, jint, jobjectArray, jintArray);
+
+/*
+ * Class:     org_apache_hadoop_io_erasurecode_rawcoder_NativeRSRawEncoder
+ * Method:    destroyImpl
+ * Signature: ()V
+ */
+JNIEXPORT void JNICALL Java_org_apache_hadoop_io_erasurecode_rawcoder_NativeRSRawEncoder_destroyImpl
+  (JNIEnv *, jobject);
+
+#ifdef __cplusplus
+}
+#endif
+#endif

http://git-wip-us.apache.org/repos/asf/hadoop/blob/658ee95f/hadoop-common-project/hadoop-common/src/main/native/src/test/org/apache/hadoop/io/erasurecode/erasure_code_test.c
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/native/src/test/org/apache/hadoop/io/erasurecode/erasure_code_test.c b/hadoop-common-project/hadoop-common/src/main/native/src/test/org/apache/hadoop/io/erasurecode/erasure_code_test.c
index 9817a76..331bb21 100644
--- a/hadoop-common-project/hadoop-common/src/main/native/src/test/org/apache/hadoop/io/erasurecode/erasure_code_test.c
+++ b/hadoop-common-project/hadoop-common/src/main/native/src/test/org/apache/hadoop/io/erasurecode/erasure_code_test.c
@@ -17,185 +17,36 @@
  */
 
 /**
- * This is a lightweight version of the same file in Intel ISA-L library to test
- * and verify the basic functions of ISA-L integration. Note it's not serving as
- * a complete ISA-L library test nor as any sample to write an erasure coder
- * using the library. A sample is to be written and provided separately.
+ * This is a sample program illustrating how to use the Intel ISA-L library.
+ * Note it's adapted from erasure_code_test.c test program, but trying to use
+ * variable names and styles we're more familiar with already similar to Java
+ * coders.
  */
 
-#include "org_apache_hadoop.h"
+#include "isal_load.h"
 #include "erasure_code.h"
 #include "gf_util.h"
+#include "erasure_coder.h"
 
 #include <stdio.h>
 #include <stdlib.h>
 #include <string.h>
 
-#define TEST_LEN 8192
-#define TEST_SOURCES  127
-#define MMAX TEST_SOURCES
-#define KMAX TEST_SOURCES
-#define TEST_SEED 11
-
-static void dump(unsigned char *buf, int len)
-{
-  int i;
-  for (i = 0; i < len;) {
-    printf(" %2x", 0xff & buf[i++]);
-    if (i % 32 == 0)
-      printf("\n");
-  }
-  printf("\n");
-}
-
-static void dump_matrix(unsigned char **s, int k, int m)
-{
+int main(int argc, char *argv[]) {
   int i, j;
-  for (i = 0; i < k; i++) {
-    for (j = 0; j < m; j++) {
-      printf(" %2x", s[i][j]);
-    }
-    printf("\n");
-  }
-  printf("\n");
-}
-
-static void dump_u8xu8(unsigned char *s, int k, int m)
-{
-  int i, j;
-  for (i = 0; i < k; i++) {
-    for (j = 0; j < m; j++) {
-      printf(" %2x", 0xff & s[j + (i * m)]);
-    }
-    printf("\n");
-  }
-  printf("\n");
-}
-
-// Generate Random errors
-static void gen_err_list(unsigned char *src_err_list,
-       unsigned char *src_in_err, int *pnerrs, int *pnsrcerrs, int k, int m)
-{
-  int i, err;
-  int nerrs = 0, nsrcerrs = 0;
-
-  for (i = 0, nerrs = 0, nsrcerrs = 0; i < m && nerrs < m - k; i++) {
-    err = 1 & rand();
-    src_in_err[i] = err;
-    if (err) {
-      src_err_list[nerrs++] = i;
-      if (i < k) {
-        nsrcerrs++;
-      }
-    }
-  }
-  if (nerrs == 0) { // should have at least one error
-    while ((err = (rand() % KMAX)) >= m) ;
-    src_err_list[nerrs++] = err;
-    src_in_err[err] = 1;
-    if (err < k)
-      nsrcerrs = 1;
-  }
-  *pnerrs = nerrs;
-  *pnsrcerrs = nsrcerrs;
-  return;
-}
-
-#define NO_INVERT_MATRIX -2
-// Generate decode matrix from encode matrix
-static int gf_gen_decode_matrix(unsigned char *encode_matrix,
-        unsigned char *decode_matrix,
-        unsigned char *invert_matrix,
-        unsigned int *decode_index,
-        unsigned char *src_err_list,
-        unsigned char *src_in_err,
-        int nerrs, int nsrcerrs, int k, int m)
-{
-  int i, j, p;
-  int r;
-  unsigned char *backup, *b, s;
-  int incr = 0;
-
-  b = malloc(MMAX * KMAX);
-  backup = malloc(MMAX * KMAX);
-
-  if (b == NULL || backup == NULL) {
-    printf("Test failure! Error with malloc\n");
-    free(b);
-    free(backup);
-    return -1;
-  }
-  // Construct matrix b by removing error rows
-  for (i = 0, r = 0; i < k; i++, r++) {
-    while (src_in_err[r])
-      r++;
-    for (j = 0; j < k; j++) {
-      b[k * i + j] = encode_matrix[k * r + j];
-      backup[k * i + j] = encode_matrix[k * r + j];
-    }
-    decode_index[i] = r;
-  }
-  incr = 0;
-  while (h_gf_invert_matrix(b, invert_matrix, k) < 0) {
-    if (nerrs == (m - k)) {
-      free(b);
-      free(backup);
-      printf("BAD MATRIX\n");
-      return NO_INVERT_MATRIX;
-    }
-    incr++;
-    memcpy(b, backup, MMAX * KMAX);
-    for (i = nsrcerrs; i < nerrs - nsrcerrs; i++) {
-      if (src_err_list[i] == (decode_index[k - 1] + incr)) {
-        // skip the erased parity line
-        incr++;
-        continue;
-      }
-    }
-    if (decode_index[k - 1] + incr >= m) {
-      free(b);
-      free(backup);
-      printf("BAD MATRIX\n");
-      return NO_INVERT_MATRIX;
-    }
-    decode_index[k - 1] += incr;
-    for (j = 0; j < k; j++)
-      b[k * (k - 1) + j] = encode_matrix[k * decode_index[k - 1] + j];
-
-  };
-
-  for (i = 0; i < nsrcerrs; i++) {
-    for (j = 0; j < k; j++) {
-      decode_matrix[k * i + j] = invert_matrix[k * src_err_list[i] + j];
-    }
-  }
-  /* src_err_list from encode_matrix * invert of b for parity decoding */
-  for (p = nsrcerrs; p < nerrs; p++) {
-    for (i = 0; i < k; i++) {
-      s = 0;
-      for (j = 0; j < k; j++)
-        s ^= h_gf_mul(invert_matrix[j * k + i],
-              encode_matrix[k * src_err_list[p] + j]);
-
-      decode_matrix[k * p + i] = s;
-    }
-  }
-  free(b);
-  free(backup);
-  return 0;
-}
-
-int main(int argc, char *argv[])
-{
   char err[256];
   size_t err_len = sizeof(err);
-  int re, i, j, p, m, k;
-  int nerrs, nsrcerrs;
-  unsigned int decode_index[MMAX];
-  unsigned char *temp_buffs[TEST_SOURCES], *buffs[TEST_SOURCES];
-  unsigned char *encode_matrix, *decode_matrix, *invert_matrix, *g_tbls;
-  unsigned char src_in_err[TEST_SOURCES], src_err_list[TEST_SOURCES];
-  unsigned char *recov[TEST_SOURCES];
+  int chunkSize = 1024;
+  int numDataUnits = 6;
+  int numParityUnits = 3;
+  unsigned char** dataUnits;
+  unsigned char** parityUnits;
+  IsalEncoder* pEncoder;
+  int erasedIndexes[2];
+  unsigned char* allUnits[MMAX];
+  IsalDecoder* pDecoder;
+  unsigned char* decodingOutput[2];
+  unsigned char** backupUnits;
 
   if (0 == build_support_erasurecode()) {
     printf("The native library isn't available, skipping this test\n");
@@ -209,102 +60,66 @@ int main(int argc, char *argv[])
   }
 
   printf("Performing erasure code test\n");
-  srand(TEST_SEED);
 
-  // Allocate the arrays
-  for (i = 0; i < TEST_SOURCES; i++) {
-    buffs[i] = malloc(TEST_LEN);
-  }
+  dataUnits = calloc(numDataUnits, sizeof(unsigned char*));
+  parityUnits = calloc(numParityUnits, sizeof(unsigned char*));
+  backupUnits = calloc(numParityUnits, sizeof(unsigned char*));
 
-  for (i = 0; i < TEST_SOURCES; i++) {
-    temp_buffs[i] = malloc(TEST_LEN);
+  // Allocate and generate data units
+  srand(135);
+  for (i = 0; i < numDataUnits; i++) {
+    dataUnits[i] = calloc(chunkSize, sizeof(unsigned char));
+    for (j = 0; j < chunkSize; j++) {
+      dataUnits[i][j] = rand();
+    }
   }
 
-  // Test erasure code by encode and recovery
-
-  encode_matrix = malloc(MMAX * KMAX);
-  decode_matrix = malloc(MMAX * KMAX);
-  invert_matrix = malloc(MMAX * KMAX);
-  g_tbls = malloc(KMAX * TEST_SOURCES * 32);
-  if (encode_matrix == NULL || decode_matrix == NULL
-      || invert_matrix == NULL || g_tbls == NULL) {
-    snprintf(err, err_len, "%s", "allocating test matrix buffers error");
-    return -1;
+  // Allocate and initialize parity units
+  for (i = 0; i < numParityUnits; i++) {
+    parityUnits[i] = calloc(chunkSize, sizeof(unsigned char));
+    for (j = 0; j < chunkSize; j++) {
+      parityUnits[i][j] = 0;
+    }
   }
 
-  m = 9;
-  k = 5;
-  if (m > MMAX || k > KMAX)
-    return -1;
-
-  // Make random data
-  for (i = 0; i < k; i++)
-    for (j = 0; j < TEST_LEN; j++)
-      buffs[i][j] = rand();
+  pEncoder = (IsalEncoder*)malloc(sizeof(IsalEncoder));
+  memset(pEncoder, 0, sizeof(*pEncoder));
+  initEncoder(pEncoder, numDataUnits, numParityUnits);
+  encode(pEncoder, dataUnits, parityUnits, chunkSize);
 
-  // The matrix generated by gf_gen_cauchy1_matrix
-  // is always invertable.
-  h_gf_gen_cauchy_matrix(encode_matrix, m, k);
+  pDecoder = (IsalDecoder*)malloc(sizeof(IsalDecoder));
+  memset(pDecoder, 0, sizeof(*pDecoder));
+  initDecoder(pDecoder, numDataUnits, numParityUnits);
 
-  // Generate g_tbls from encode matrix encode_matrix
-  h_ec_init_tables(k, m - k, &encode_matrix[k * k], g_tbls);
+  memcpy(allUnits, dataUnits, numDataUnits * (sizeof (unsigned char*)));
+  memcpy(allUnits + numDataUnits, parityUnits,
+                            numParityUnits * (sizeof (unsigned char*)));
 
-  // Perform matrix dot_prod for EC encoding
-  // using g_tbls from encode matrix encode_matrix
-  h_ec_encode_data(TEST_LEN, k, m - k, g_tbls, buffs, &buffs[k]);
+  erasedIndexes[0] = 1;
+  erasedIndexes[1] = 7;
 
-  // Choose random buffers to be in erasure
-  memset(src_in_err, 0, TEST_SOURCES);
-  gen_err_list(src_err_list, src_in_err, &nerrs, &nsrcerrs, k, m);
+  backupUnits[0] = allUnits[1];
+  backupUnits[1] = allUnits[7];
 
-  // Generate decode matrix
-  re = gf_gen_decode_matrix(encode_matrix, decode_matrix,
-          invert_matrix, decode_index, src_err_list, src_in_err,
-          nerrs, nsrcerrs, k, m);
-  if (re != 0) {
-    snprintf(err, err_len, "%s", "gf_gen_decode_matrix failed");
-    return -1;
-  }
-  // Pack recovery array as list of valid sources
-  // Its order must be the same as the order
-  // to generate matrix b in gf_gen_decode_matrix
-  for (i = 0; i < k; i++) {
-    recov[i] = buffs[decode_index[i]];
-  }
+  allUnits[0] = NULL; // Not to read
+  allUnits[1] = NULL;
+  allUnits[7] = NULL;
 
-  // Recover data
-  h_ec_init_tables(k, nerrs, decode_matrix, g_tbls);
-  h_ec_encode_data(TEST_LEN, k, nerrs, g_tbls, recov, &temp_buffs[k]);
-  for (i = 0; i < nerrs; i++) {
-    if (0 != memcmp(temp_buffs[k + i], buffs[src_err_list[i]], TEST_LEN)) {
-      snprintf(err, err_len, "%s", "Error recovery failed");
-      printf("Fail error recovery (%d, %d, %d)\n", m, k, nerrs);
+  decodingOutput[0] = malloc(chunkSize);
+  decodingOutput[1] = malloc(chunkSize);
 
-      printf(" - erase list = ");
-      for (j = 0; j < nerrs; j++) {
-        printf(" %d", src_err_list[j]);
-      }
-
-      printf(" - Index = ");
-      for (p = 0; p < k; p++) {
-        printf(" %d", decode_index[p]);
-      }
-
-      printf("\nencode_matrix:\n");
-      dump_u8xu8((unsigned char *) encode_matrix, m, k);
-      printf("inv b:\n");
-      dump_u8xu8((unsigned char *) invert_matrix, k, k);
-      printf("\ndecode_matrix:\n");
-      dump_u8xu8((unsigned char *) decode_matrix, m, k);
-      printf("recov %d:", src_err_list[i]);
-      dump(temp_buffs[k + i], 25);
-      printf("orig   :");
-      dump(buffs[src_err_list[i]], 25);
+  decode(pDecoder, allUnits, erasedIndexes, 2, decodingOutput, chunkSize);
 
+  for (i = 0; i < pDecoder->numErased; i++) {
+    if (0 != memcmp(decodingOutput[i], backupUnits[i], chunkSize)) {
+      fprintf(stderr, "Decoding failed\n\n");
+      dumpDecoder(pDecoder);
       return -1;
     }
   }
 
-  printf("done EC tests: Pass\n");
+  dumpDecoder(pDecoder);
+  fprintf(stdout, "Successfully done, passed!\n\n");
+
   return 0;
 }


[15/46] hadoop git commit: HADOOP-12826. Rename the new Java coder and make it default. Contributed by Rui Li.

Posted by ae...@apache.org.
HADOOP-12826. Rename the new Java coder and make it default. Contributed by Rui Li.


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

Branch: refs/heads/HDFS-7240
Commit: 19e8f076919932b17f24ec4090df1926677651e7
Parents: 1898810
Author: Zhe Zhang <ze...@zezhang-ld1.linkedin.biz>
Authored: Mon Mar 14 16:45:32 2016 -0700
Committer: Zhe Zhang <ze...@zezhang-ld1.linkedin.biz>
Committed: Mon Mar 14 16:45:32 2016 -0700

----------------------------------------------------------------------
 .../apache/hadoop/io/erasurecode/CodecUtil.java |  13 +-
 .../io/erasurecode/rawcoder/RSRawDecoder.java   | 175 ++++++++++++++++++
 .../io/erasurecode/rawcoder/RSRawDecoder2.java  | 176 -------------------
 .../io/erasurecode/rawcoder/RSRawEncoder.java   |  75 ++++++++
 .../io/erasurecode/rawcoder/RSRawEncoder2.java  |  76 --------
 .../rawcoder/RSRawErasureCoderFactory.java      |  37 ++++
 .../rawcoder/RSRawErasureCoderFactory2.java     |  37 ----
 .../io/erasurecode/rawcoder/util/RSUtil.java    | 149 +++++++++++++++-
 .../io/erasurecode/rawcoder/util/RSUtil2.java   | 172 ------------------
 .../coder/TestHHXORErasureCoder.java            |   4 +-
 .../erasurecode/coder/TestRSErasureCoder.java   |   8 +-
 .../io/erasurecode/rawcoder/TestRSRawCoder.java |  33 ++++
 .../erasurecode/rawcoder/TestRSRawCoder2.java   |  33 ----
 13 files changed, 483 insertions(+), 505 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/19e8f076/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/CodecUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/CodecUtil.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/CodecUtil.java
index fd5bd67..a2354b6 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/CodecUtil.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/CodecUtil.java
@@ -20,7 +20,14 @@ package org.apache.hadoop.io.erasurecode;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.CommonConfigurationKeys;
-import org.apache.hadoop.io.erasurecode.rawcoder.*;
+import org.apache.hadoop.io.erasurecode.rawcoder.RSRawDecoder;
+import org.apache.hadoop.io.erasurecode.rawcoder.RSRawEncoder;
+import org.apache.hadoop.io.erasurecode.rawcoder.RawErasureCoder;
+import org.apache.hadoop.io.erasurecode.rawcoder.RawErasureCoderFactory;
+import org.apache.hadoop.io.erasurecode.rawcoder.RawErasureDecoder;
+import org.apache.hadoop.io.erasurecode.rawcoder.RawErasureEncoder;
+import org.apache.hadoop.io.erasurecode.rawcoder.XORRawDecoder;
+import org.apache.hadoop.io.erasurecode.rawcoder.XORRawEncoder;
 
 /**
  * A codec & coder utility to help create raw coders conveniently.
@@ -43,7 +50,7 @@ public final class CodecUtil {
         CommonConfigurationKeys.IO_ERASURECODE_CODEC_RS_RAWCODER_KEY,
         true, numDataUnits, numParityUnits);
     if (rawCoder == null) {
-      rawCoder = new RSRawEncoderLegacy(numDataUnits, numParityUnits);
+      rawCoder = new RSRawEncoder(numDataUnits, numParityUnits);
     }
 
     return (RawErasureEncoder) rawCoder;
@@ -62,7 +69,7 @@ public final class CodecUtil {
         CommonConfigurationKeys.IO_ERASURECODE_CODEC_RS_RAWCODER_KEY,
         false, numDataUnits, numParityUnits);
     if (rawCoder == null) {
-      rawCoder = new RSRawDecoderLegacy(numDataUnits, numParityUnits);
+      rawCoder = new RSRawDecoder(numDataUnits, numParityUnits);
     }
 
     return (RawErasureDecoder) rawCoder;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/19e8f076/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RSRawDecoder.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RSRawDecoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RSRawDecoder.java
new file mode 100644
index 0000000..5b9e0e9
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RSRawDecoder.java
@@ -0,0 +1,175 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.io.erasurecode.rawcoder;
+
+import org.apache.hadoop.HadoopIllegalArgumentException;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.io.erasurecode.rawcoder.util.CoderUtil;
+import org.apache.hadoop.io.erasurecode.rawcoder.util.DumpUtil;
+import org.apache.hadoop.io.erasurecode.rawcoder.util.GF256;
+import org.apache.hadoop.io.erasurecode.rawcoder.util.RSUtil;
+
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+
+/**
+ * A raw erasure decoder in RS code scheme in pure Java in case native one
+ * isn't available in some environment. Please always use native implementations
+ * when possible. This new Java coder is about 5X faster than the one originated
+ * from HDFS-RAID, and also compatible with the native/ISA-L coder.
+ */
+@InterfaceAudience.Private
+public class RSRawDecoder extends AbstractRawErasureDecoder {
+  //relevant to schema and won't change during decode calls
+  private byte[] encodeMatrix;
+
+  /**
+   * Below are relevant to schema and erased indexes, thus may change during
+   * decode calls.
+   */
+  private byte[] decodeMatrix;
+  private byte[] invertMatrix;
+  /**
+   * Array of input tables generated from coding coefficients previously.
+   * Must be of size 32*k*rows
+   */
+  private byte[] gfTables;
+  private int[] cachedErasedIndexes;
+  private int[] validIndexes;
+  private int numErasedDataUnits;
+  private boolean[] erasureFlags;
+
+  public RSRawDecoder(int numDataUnits, int numParityUnits) {
+    super(numDataUnits, numParityUnits);
+    if (numDataUnits + numParityUnits >= RSUtil.GF.getFieldSize()) {
+      throw new HadoopIllegalArgumentException(
+              "Invalid getNumDataUnits() and numParityUnits");
+    }
+
+    int numAllUnits = getNumDataUnits() + numParityUnits;
+    encodeMatrix = new byte[numAllUnits * getNumDataUnits()];
+    RSUtil.genCauchyMatrix(encodeMatrix, numAllUnits, getNumDataUnits());
+    if (isAllowingVerboseDump()) {
+      DumpUtil.dumpMatrix(encodeMatrix, numDataUnits, numAllUnits);
+    }
+  }
+
+  @Override
+  protected void doDecode(ByteBuffer[] inputs, int[] erasedIndexes,
+                          ByteBuffer[] outputs) {
+    prepareDecoding(inputs, erasedIndexes);
+
+    ByteBuffer[] realInputs = new ByteBuffer[getNumDataUnits()];
+    for (int i = 0; i < getNumDataUnits(); i++) {
+      realInputs[i] = inputs[validIndexes[i]];
+    }
+    RSUtil.encodeData(gfTables, realInputs, outputs);
+  }
+
+  @Override
+  protected void doDecode(byte[][] inputs, int[] inputOffsets,
+                          int dataLen, int[] erasedIndexes,
+                          byte[][] outputs, int[] outputOffsets) {
+    prepareDecoding(inputs, erasedIndexes);
+
+    byte[][] realInputs = new byte[getNumDataUnits()][];
+    int[] realInputOffsets = new int[getNumDataUnits()];
+    for (int i = 0; i < getNumDataUnits(); i++) {
+      realInputs[i] = inputs[validIndexes[i]];
+      realInputOffsets[i] = inputOffsets[validIndexes[i]];
+    }
+    RSUtil.encodeData(gfTables, dataLen, realInputs, realInputOffsets,
+            outputs, outputOffsets);
+  }
+
+  private <T> void prepareDecoding(T[] inputs, int[] erasedIndexes) {
+    int[] tmpValidIndexes = new int[getNumDataUnits()];
+    CoderUtil.makeValidIndexes(inputs, tmpValidIndexes);
+    if (Arrays.equals(this.cachedErasedIndexes, erasedIndexes) &&
+        Arrays.equals(this.validIndexes, tmpValidIndexes)) {
+      return; // Optimization. Nothing to do
+    }
+    this.cachedErasedIndexes =
+            Arrays.copyOf(erasedIndexes, erasedIndexes.length);
+    this.validIndexes =
+            Arrays.copyOf(tmpValidIndexes, tmpValidIndexes.length);
+
+    processErasures(erasedIndexes);
+  }
+
+  private void processErasures(int[] erasedIndexes) {
+    this.decodeMatrix = new byte[getNumAllUnits() * getNumDataUnits()];
+    this.invertMatrix = new byte[getNumAllUnits() * getNumDataUnits()];
+    this.gfTables = new byte[getNumAllUnits() * getNumDataUnits() * 32];
+
+    this.erasureFlags = new boolean[getNumAllUnits()];
+    this.numErasedDataUnits = 0;
+
+    for (int i = 0; i < erasedIndexes.length; i++) {
+      int index = erasedIndexes[i];
+      erasureFlags[index] = true;
+      if (index < getNumDataUnits()) {
+        numErasedDataUnits++;
+      }
+    }
+
+    generateDecodeMatrix(erasedIndexes);
+
+    RSUtil.initTables(getNumDataUnits(), erasedIndexes.length,
+        decodeMatrix, 0, gfTables);
+    if (isAllowingVerboseDump()) {
+      System.out.println(DumpUtil.bytesToHex(gfTables, -1));
+    }
+  }
+
+  // Generate decode matrix from encode matrix
+  private void generateDecodeMatrix(int[] erasedIndexes) {
+    int i, j, r, p;
+    byte s;
+    byte[] tmpMatrix = new byte[getNumAllUnits() * getNumDataUnits()];
+
+    // Construct matrix tmpMatrix by removing error rows
+    for (i = 0; i < getNumDataUnits(); i++) {
+      r = validIndexes[i];
+      for (j = 0; j < getNumDataUnits(); j++) {
+        tmpMatrix[getNumDataUnits() * i + j] =
+                encodeMatrix[getNumDataUnits() * r + j];
+      }
+    }
+
+    GF256.gfInvertMatrix(tmpMatrix, invertMatrix, getNumDataUnits());
+
+    for (i = 0; i < numErasedDataUnits; i++) {
+      for (j = 0; j < getNumDataUnits(); j++) {
+        decodeMatrix[getNumDataUnits() * i + j] =
+                invertMatrix[getNumDataUnits() * erasedIndexes[i] + j];
+      }
+    }
+
+    for (p = numErasedDataUnits; p < erasedIndexes.length; p++) {
+      for (i = 0; i < getNumDataUnits(); i++) {
+        s = 0;
+        for (j = 0; j < getNumDataUnits(); j++) {
+          s ^= GF256.gfMul(invertMatrix[j * getNumDataUnits() + i],
+                  encodeMatrix[getNumDataUnits() * erasedIndexes[p] + j]);
+        }
+        decodeMatrix[getNumDataUnits() * p + i] = s;
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/19e8f076/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RSRawDecoder2.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RSRawDecoder2.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RSRawDecoder2.java
deleted file mode 100644
index 48a3579..0000000
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RSRawDecoder2.java
+++ /dev/null
@@ -1,176 +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.io.erasurecode.rawcoder;
-
-import org.apache.hadoop.HadoopIllegalArgumentException;
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.io.erasurecode.rawcoder.util.CoderUtil;
-import org.apache.hadoop.io.erasurecode.rawcoder.util.DumpUtil;
-import org.apache.hadoop.io.erasurecode.rawcoder.util.GF256;
-import org.apache.hadoop.io.erasurecode.rawcoder.util.RSUtil;
-import org.apache.hadoop.io.erasurecode.rawcoder.util.RSUtil2;
-
-import java.nio.ByteBuffer;
-import java.util.Arrays;
-
-/**
- * A raw erasure decoder in RS code scheme in pure Java in case native one
- * isn't available in some environment. Please always use native implementations
- * when possible. This new Java coder is about 5X faster than the one originated
- * from HDFS-RAID, and also compatible with the native/ISA-L coder.
- */
-@InterfaceAudience.Private
-public class RSRawDecoder2 extends AbstractRawErasureDecoder {
-  //relevant to schema and won't change during decode calls
-  private byte[] encodeMatrix;
-
-  /**
-   * Below are relevant to schema and erased indexes, thus may change during
-   * decode calls.
-   */
-  private byte[] decodeMatrix;
-  private byte[] invertMatrix;
-  /**
-   * Array of input tables generated from coding coefficients previously.
-   * Must be of size 32*k*rows
-   */
-  private byte[] gfTables;
-  private int[] cachedErasedIndexes;
-  private int[] validIndexes;
-  private int numErasedDataUnits;
-  private boolean[] erasureFlags;
-
-  public RSRawDecoder2(int numDataUnits, int numParityUnits) {
-    super(numDataUnits, numParityUnits);
-    if (numDataUnits + numParityUnits >= RSUtil.GF.getFieldSize()) {
-      throw new HadoopIllegalArgumentException(
-              "Invalid getNumDataUnits() and numParityUnits");
-    }
-
-    int numAllUnits = getNumDataUnits() + numParityUnits;
-    encodeMatrix = new byte[numAllUnits * getNumDataUnits()];
-    RSUtil2.genCauchyMatrix(encodeMatrix, numAllUnits, getNumDataUnits());
-    if (isAllowingVerboseDump()) {
-      DumpUtil.dumpMatrix(encodeMatrix, numDataUnits, numAllUnits);
-    }
-  }
-
-  @Override
-  protected void doDecode(ByteBuffer[] inputs, int[] erasedIndexes,
-                          ByteBuffer[] outputs) {
-    prepareDecoding(inputs, erasedIndexes);
-
-    ByteBuffer[] realInputs = new ByteBuffer[getNumDataUnits()];
-    for (int i = 0; i < getNumDataUnits(); i++) {
-      realInputs[i] = inputs[validIndexes[i]];
-    }
-    RSUtil2.encodeData(gfTables, realInputs, outputs);
-  }
-
-  @Override
-  protected void doDecode(byte[][] inputs, int[] inputOffsets,
-                          int dataLen, int[] erasedIndexes,
-                          byte[][] outputs, int[] outputOffsets) {
-    prepareDecoding(inputs, erasedIndexes);
-
-    byte[][] realInputs = new byte[getNumDataUnits()][];
-    int[] realInputOffsets = new int[getNumDataUnits()];
-    for (int i = 0; i < getNumDataUnits(); i++) {
-      realInputs[i] = inputs[validIndexes[i]];
-      realInputOffsets[i] = inputOffsets[validIndexes[i]];
-    }
-    RSUtil2.encodeData(gfTables, dataLen, realInputs, realInputOffsets,
-            outputs, outputOffsets);
-  }
-
-  private <T> void prepareDecoding(T[] inputs, int[] erasedIndexes) {
-    int[] tmpValidIndexes = new int[getNumDataUnits()];
-    CoderUtil.makeValidIndexes(inputs, tmpValidIndexes);
-    if (Arrays.equals(this.cachedErasedIndexes, erasedIndexes) &&
-        Arrays.equals(this.validIndexes, tmpValidIndexes)) {
-      return; // Optimization. Nothing to do
-    }
-    this.cachedErasedIndexes =
-            Arrays.copyOf(erasedIndexes, erasedIndexes.length);
-    this.validIndexes =
-            Arrays.copyOf(tmpValidIndexes, tmpValidIndexes.length);
-
-    processErasures(erasedIndexes);
-  }
-
-  private void processErasures(int[] erasedIndexes) {
-    this.decodeMatrix = new byte[getNumAllUnits() * getNumDataUnits()];
-    this.invertMatrix = new byte[getNumAllUnits() * getNumDataUnits()];
-    this.gfTables = new byte[getNumAllUnits() * getNumDataUnits() * 32];
-
-    this.erasureFlags = new boolean[getNumAllUnits()];
-    this.numErasedDataUnits = 0;
-
-    for (int i = 0; i < erasedIndexes.length; i++) {
-      int index = erasedIndexes[i];
-      erasureFlags[index] = true;
-      if (index < getNumDataUnits()) {
-        numErasedDataUnits++;
-      }
-    }
-
-    generateDecodeMatrix(erasedIndexes);
-
-    RSUtil2.initTables(getNumDataUnits(), erasedIndexes.length,
-        decodeMatrix, 0, gfTables);
-    if (isAllowingVerboseDump()) {
-      System.out.println(DumpUtil.bytesToHex(gfTables, -1));
-    }
-  }
-
-  // Generate decode matrix from encode matrix
-  private void generateDecodeMatrix(int[] erasedIndexes) {
-    int i, j, r, p;
-    byte s;
-    byte[] tmpMatrix = new byte[getNumAllUnits() * getNumDataUnits()];
-
-    // Construct matrix tmpMatrix by removing error rows
-    for (i = 0; i < getNumDataUnits(); i++) {
-      r = validIndexes[i];
-      for (j = 0; j < getNumDataUnits(); j++) {
-        tmpMatrix[getNumDataUnits() * i + j] =
-                encodeMatrix[getNumDataUnits() * r + j];
-      }
-    }
-
-    GF256.gfInvertMatrix(tmpMatrix, invertMatrix, getNumDataUnits());
-
-    for (i = 0; i < numErasedDataUnits; i++) {
-      for (j = 0; j < getNumDataUnits(); j++) {
-        decodeMatrix[getNumDataUnits() * i + j] =
-                invertMatrix[getNumDataUnits() * erasedIndexes[i] + j];
-      }
-    }
-
-    for (p = numErasedDataUnits; p < erasedIndexes.length; p++) {
-      for (i = 0; i < getNumDataUnits(); i++) {
-        s = 0;
-        for (j = 0; j < getNumDataUnits(); j++) {
-          s ^= GF256.gfMul(invertMatrix[j * getNumDataUnits() + i],
-                  encodeMatrix[getNumDataUnits() * erasedIndexes[p] + j]);
-        }
-        decodeMatrix[getNumDataUnits() * p + i] = s;
-      }
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/19e8f076/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RSRawEncoder.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RSRawEncoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RSRawEncoder.java
new file mode 100644
index 0000000..cee6574
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RSRawEncoder.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.io.erasurecode.rawcoder;
+
+import org.apache.hadoop.HadoopIllegalArgumentException;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.io.erasurecode.rawcoder.util.DumpUtil;
+import org.apache.hadoop.io.erasurecode.rawcoder.util.RSUtil;
+
+import java.nio.ByteBuffer;
+
+/**
+ * A raw erasure encoder in RS code scheme in pure Java in case native one
+ * isn't available in some environment. Please always use native implementations
+ * when possible. This new Java coder is about 5X faster than the one originated
+ * from HDFS-RAID, and also compatible with the native/ISA-L coder.
+ */
+@InterfaceAudience.Private
+public class RSRawEncoder extends AbstractRawErasureEncoder {
+  // relevant to schema and won't change during encode calls.
+  private byte[] encodeMatrix;
+  /**
+   * Array of input tables generated from coding coefficients previously.
+   * Must be of size 32*k*rows
+   */
+  private byte[] gfTables;
+
+  public RSRawEncoder(int numDataUnits, int numParityUnits) {
+    super(numDataUnits, numParityUnits);
+
+    if (numDataUnits + numParityUnits >= RSUtil.GF.getFieldSize()) {
+      throw new HadoopIllegalArgumentException(
+          "Invalid numDataUnits and numParityUnits");
+    }
+
+    encodeMatrix = new byte[getNumAllUnits() * numDataUnits];
+    RSUtil.genCauchyMatrix(encodeMatrix, getNumAllUnits(), numDataUnits);
+    if (isAllowingVerboseDump()) {
+      DumpUtil.dumpMatrix(encodeMatrix, numDataUnits, getNumAllUnits());
+    }
+    gfTables = new byte[getNumAllUnits() * numDataUnits * 32];
+    RSUtil.initTables(numDataUnits, numParityUnits, encodeMatrix,
+        numDataUnits * numDataUnits, gfTables);
+    if (isAllowingVerboseDump()) {
+      System.out.println(DumpUtil.bytesToHex(gfTables, -1));
+    }
+  }
+
+  @Override
+  protected void doEncode(ByteBuffer[] inputs, ByteBuffer[] outputs) {
+    RSUtil.encodeData(gfTables, inputs, outputs);
+  }
+
+  @Override
+  protected void doEncode(byte[][] inputs, int[] inputOffsets,
+                          int dataLen, byte[][] outputs, int[] outputOffsets) {
+    RSUtil.encodeData(gfTables, dataLen, inputs, inputOffsets, outputs,
+        outputOffsets);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/19e8f076/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RSRawEncoder2.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RSRawEncoder2.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RSRawEncoder2.java
deleted file mode 100644
index 72d77f7..0000000
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RSRawEncoder2.java
+++ /dev/null
@@ -1,76 +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.io.erasurecode.rawcoder;
-
-import org.apache.hadoop.HadoopIllegalArgumentException;
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.io.erasurecode.rawcoder.util.DumpUtil;
-import org.apache.hadoop.io.erasurecode.rawcoder.util.RSUtil;
-import org.apache.hadoop.io.erasurecode.rawcoder.util.RSUtil2;
-
-import java.nio.ByteBuffer;
-
-/**
- * A raw erasure encoder in RS code scheme in pure Java in case native one
- * isn't available in some environment. Please always use native implementations
- * when possible. This new Java coder is about 5X faster than the one originated
- * from HDFS-RAID, and also compatible with the native/ISA-L coder.
- */
-@InterfaceAudience.Private
-public class RSRawEncoder2 extends AbstractRawErasureEncoder {
-  // relevant to schema and won't change during encode calls.
-  private byte[] encodeMatrix;
-  /**
-   * Array of input tables generated from coding coefficients previously.
-   * Must be of size 32*k*rows
-   */
-  private byte[] gfTables;
-
-  public RSRawEncoder2(int numDataUnits, int numParityUnits) {
-    super(numDataUnits, numParityUnits);
-
-    if (numDataUnits + numParityUnits >= RSUtil.GF.getFieldSize()) {
-      throw new HadoopIllegalArgumentException(
-          "Invalid numDataUnits and numParityUnits");
-    }
-
-    encodeMatrix = new byte[getNumAllUnits() * numDataUnits];
-    RSUtil2.genCauchyMatrix(encodeMatrix, getNumAllUnits(), numDataUnits);
-    if (isAllowingVerboseDump()) {
-      DumpUtil.dumpMatrix(encodeMatrix, numDataUnits, getNumAllUnits());
-    }
-    gfTables = new byte[getNumAllUnits() * numDataUnits * 32];
-    RSUtil2.initTables(numDataUnits, numParityUnits, encodeMatrix,
-        numDataUnits * numDataUnits, gfTables);
-    if (isAllowingVerboseDump()) {
-      System.out.println(DumpUtil.bytesToHex(gfTables, -1));
-    }
-  }
-
-  @Override
-  protected void doEncode(ByteBuffer[] inputs, ByteBuffer[] outputs) {
-    RSUtil2.encodeData(gfTables, inputs, outputs);
-  }
-
-  @Override
-  protected void doEncode(byte[][] inputs, int[] inputOffsets,
-                          int dataLen, byte[][] outputs, int[] outputOffsets) {
-    RSUtil2.encodeData(gfTables, dataLen, inputs, inputOffsets, outputs,
-        outputOffsets);
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/19e8f076/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RSRawErasureCoderFactory.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RSRawErasureCoderFactory.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RSRawErasureCoderFactory.java
new file mode 100644
index 0000000..b38db4b
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RSRawErasureCoderFactory.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.io.erasurecode.rawcoder;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+
+/**
+ * A raw coder factory for the new raw Reed-Solomon coder in Java.
+ */
+@InterfaceAudience.Private
+public class RSRawErasureCoderFactory implements RawErasureCoderFactory {
+
+  @Override
+  public RawErasureEncoder createEncoder(int numDataUnits, int numParityUnits) {
+    return new RSRawEncoder(numDataUnits, numParityUnits);
+  }
+
+  @Override
+  public RawErasureDecoder createDecoder(int numDataUnits, int numParityUnits) {
+    return new RSRawDecoder(numDataUnits, numParityUnits);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/19e8f076/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RSRawErasureCoderFactory2.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RSRawErasureCoderFactory2.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RSRawErasureCoderFactory2.java
deleted file mode 100644
index 40a0f1c..0000000
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RSRawErasureCoderFactory2.java
+++ /dev/null
@@ -1,37 +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.io.erasurecode.rawcoder;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-
-/**
- * A raw coder factory for raw Reed-Solomon coder in Java.
- */
-@InterfaceAudience.Private
-public class RSRawErasureCoderFactory2 implements RawErasureCoderFactory {
-
-  @Override
-  public RawErasureEncoder createEncoder(int numDataUnits, int numParityUnits) {
-    return new RSRawEncoder2(numDataUnits, numParityUnits);
-  }
-
-  @Override
-  public RawErasureDecoder createDecoder(int numDataUnits, int numParityUnits) {
-    return new RSRawDecoder2(numDataUnits, numParityUnits);
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/19e8f076/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/util/RSUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/util/RSUtil.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/util/RSUtil.java
index a3b0e39..43823d0 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/util/RSUtil.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/util/RSUtil.java
@@ -19,11 +19,16 @@ package org.apache.hadoop.io.erasurecode.rawcoder.util;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 
+import java.nio.ByteBuffer;
+
 /**
- * Utilities for implementing Reed-Solomon code, used by RS coder.
+ * Utilities for implementing Reed-Solomon code, used by RS coder. Some of the
+ * codes are borrowed from ISA-L implementation (C or ASM codes).
  */
 @InterfaceAudience.Private
-public class RSUtil {
+public final class RSUtil {
+
+  private RSUtil(){}
 
   // We always use the byte system (with symbol size 8, field size 256,
   // primitive polynomial 285, and primitive root 2).
@@ -39,4 +44,144 @@ public class RSUtil {
     return primitivePower;
   }
 
+  public static void initTables(int k, int rows, byte[] codingMatrix,
+      int matrixOffset, byte[] gfTables) {
+    int i, j;
+
+    int offset = 0, idx = matrixOffset;
+    for (i = 0; i < rows; i++) {
+      for (j = 0; j < k; j++) {
+        GF256.gfVectMulInit(codingMatrix[idx++], gfTables, offset);
+        offset += 32;
+      }
+    }
+  }
+
+  /**
+   * Ported from Intel ISA-L library.
+   */
+  public static void genCauchyMatrix(byte[] a, int m, int k) {
+    // Identity matrix in high position
+    for (int i = 0; i < k; i++) {
+      a[k * i + i] = 1;
+    }
+
+    // For the rest choose 1/(i + j) | i != j
+    int pos = k * k;
+    for (int i = k; i < m; i++) {
+      for (int j = 0; j < k; j++) {
+        a[pos++] = GF256.gfInv((byte) (i ^ j));
+      }
+    }
+  }
+
+  /**
+   * Encode a group of inputs data and generate the outputs. It's also used for
+   * decoding because, in this implementation, encoding and decoding are
+   * unified.
+   *
+   * The algorithm is ported from Intel ISA-L library for compatible. It
+   * leverages Java auto-vectorization support for performance.
+   */
+  public static void encodeData(byte[] gfTables, int dataLen, byte[][] inputs,
+      int[] inputOffsets, byte[][] outputs,
+      int[] outputOffsets) {
+    int numInputs = inputs.length;
+    int numOutputs = outputs.length;
+    int l, i, j, iPos, oPos;
+    byte[] input, output;
+    byte s;
+    final int times = dataLen / 8;
+    final int extra = dataLen - dataLen % 8;
+    byte[] tableLine;
+
+    for (l = 0; l < numOutputs; l++) {
+      output = outputs[l];
+
+      for (j = 0; j < numInputs; j++) {
+        input = inputs[j];
+        iPos = inputOffsets[j];
+        oPos = outputOffsets[l];
+
+        s = gfTables[j * 32 + l * numInputs * 32 + 1];
+        tableLine = GF256.gfMulTab()[s & 0xff];
+
+        /**
+         * Purely for performance, assuming we can use 8 bytes in the SIMD
+         * instruction. Subject to be improved.
+         */
+        for (i = 0; i < times; i++, iPos += 8, oPos += 8) {
+          output[oPos + 0] ^= tableLine[0xff & input[iPos + 0]];
+          output[oPos + 1] ^= tableLine[0xff & input[iPos + 1]];
+          output[oPos + 2] ^= tableLine[0xff & input[iPos + 2]];
+          output[oPos + 3] ^= tableLine[0xff & input[iPos + 3]];
+          output[oPos + 4] ^= tableLine[0xff & input[iPos + 4]];
+          output[oPos + 5] ^= tableLine[0xff & input[iPos + 5]];
+          output[oPos + 6] ^= tableLine[0xff & input[iPos + 6]];
+          output[oPos + 7] ^= tableLine[0xff & input[iPos + 7]];
+        }
+
+        /**
+         * For the left bytes, do it one by one.
+         */
+        for (i = extra; i < dataLen; i++, iPos++, oPos++) {
+          output[oPos] ^= tableLine[0xff & input[iPos]];
+        }
+      }
+    }
+  }
+
+  /**
+   * See above. Try to use the byte[] version when possible.
+   */
+  public static void encodeData(byte[] gfTables, ByteBuffer[] inputs,
+      ByteBuffer[] outputs) {
+    int numInputs = inputs.length;
+    int numOutputs = outputs.length;
+    int dataLen = inputs[0].remaining();
+    int l, i, j, iPos, oPos;
+    ByteBuffer input, output;
+    byte s;
+    final int times = dataLen / 8;
+    final int extra = dataLen - dataLen % 8;
+    byte[] tableLine;
+
+    for (l = 0; l < numOutputs; l++) {
+      output = outputs[l];
+
+      for (j = 0; j < numInputs; j++) {
+        input = inputs[j];
+        iPos = input.position();
+        oPos = output.position();
+
+        s = gfTables[j * 32 + l * numInputs * 32 + 1];
+        tableLine = GF256.gfMulTab()[s & 0xff];
+
+        for (i = 0; i < times; i++, iPos += 8, oPos += 8) {
+          output.put(oPos + 0, (byte) (output.get(oPos + 0) ^
+              tableLine[0xff & input.get(iPos + 0)]));
+          output.put(oPos + 1, (byte) (output.get(oPos + 1) ^
+              tableLine[0xff & input.get(iPos + 1)]));
+          output.put(oPos + 2, (byte) (output.get(oPos + 2) ^
+              tableLine[0xff & input.get(iPos + 2)]));
+          output.put(oPos + 3, (byte) (output.get(oPos + 3) ^
+              tableLine[0xff & input.get(iPos + 3)]));
+          output.put(oPos + 4, (byte) (output.get(oPos + 4) ^
+              tableLine[0xff & input.get(iPos + 4)]));
+          output.put(oPos + 5, (byte) (output.get(oPos + 5) ^
+              tableLine[0xff & input.get(iPos + 5)]));
+          output.put(oPos + 6, (byte) (output.get(oPos + 6) ^
+              tableLine[0xff & input.get(iPos + 6)]));
+          output.put(oPos + 7, (byte) (output.get(oPos + 7) ^
+              tableLine[0xff & input.get(iPos + 7)]));
+        }
+
+        for (i = extra; i < dataLen; i++, iPos++, oPos++) {
+          output.put(oPos, (byte) (output.get(oPos) ^
+              tableLine[0xff & input.get(iPos)]));
+        }
+      }
+    }
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/19e8f076/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/util/RSUtil2.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/util/RSUtil2.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/util/RSUtil2.java
deleted file mode 100644
index 84121a8..0000000
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/util/RSUtil2.java
+++ /dev/null
@@ -1,172 +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.io.erasurecode.rawcoder.util;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-
-import java.nio.ByteBuffer;
-
-/**
- * Utilities for implementing Reed-Solomon code, used by RS2 coder. Some of the
- * codes are borrowed from ISA-L implementation (C or ASM codes).
- */
-@InterfaceAudience.Private
-public final class RSUtil2 {
-
-  private RSUtil2() { }
-
-  public static void initTables(int k, int rows, byte[] codingMatrix,
-                                int matrixOffset, byte[] gfTables) {
-    int i, j;
-
-    int offset = 0, idx = matrixOffset;
-    for (i = 0; i < rows; i++) {
-      for (j = 0; j < k; j++) {
-        GF256.gfVectMulInit(codingMatrix[idx++], gfTables, offset);
-        offset += 32;
-      }
-    }
-  }
-
-  /**
-   * Ported from Intel ISA-L library.
-   */
-  public static void genCauchyMatrix(byte[] a, int m, int k) {
-    // Identity matrix in high position
-    for (int i = 0; i < k; i++) {
-      a[k * i + i] = 1;
-    }
-
-    // For the rest choose 1/(i + j) | i != j
-    int pos = k * k;
-    for (int i = k; i < m; i++) {
-      for (int j = 0; j < k; j++) {
-        a[pos++] = GF256.gfInv((byte) (i ^ j));
-      }
-    }
-  }
-
-  /**
-   * Encode a group of inputs data and generate the outputs. It's also used for
-   * decoding because, in this implementation, encoding and decoding are
-   * unified.
-   *
-   * The algorithm is ported from Intel ISA-L library for compatible. It
-   * leverages Java auto-vectorization support for performance.
-   */
-  public static void encodeData(byte[] gfTables, int dataLen, byte[][] inputs,
-                                int[] inputOffsets, byte[][] outputs,
-                                int[] outputOffsets) {
-    int numInputs = inputs.length;
-    int numOutputs = outputs.length;
-    int l, i, j, iPos, oPos;
-    byte[] input, output;
-    byte s;
-    final int times = dataLen / 8;
-    final int extra = dataLen - dataLen % 8;
-    byte[] tableLine;
-
-    for (l = 0; l < numOutputs; l++) {
-      output = outputs[l];
-
-      for (j = 0; j < numInputs; j++) {
-        input = inputs[j];
-        iPos = inputOffsets[j];
-        oPos = outputOffsets[l];
-
-        s = gfTables[j * 32 + l * numInputs * 32 + 1];
-        tableLine = GF256.gfMulTab()[s & 0xff];
-
-        /**
-         * Purely for performance, assuming we can use 8 bytes in the SIMD
-         * instruction. Subject to be improved.
-         */
-        for (i = 0; i < times; i++, iPos += 8, oPos += 8) {
-          output[oPos + 0] ^= tableLine[0xff & input[iPos + 0]];
-          output[oPos + 1] ^= tableLine[0xff & input[iPos + 1]];
-          output[oPos + 2] ^= tableLine[0xff & input[iPos + 2]];
-          output[oPos + 3] ^= tableLine[0xff & input[iPos + 3]];
-          output[oPos + 4] ^= tableLine[0xff & input[iPos + 4]];
-          output[oPos + 5] ^= tableLine[0xff & input[iPos + 5]];
-          output[oPos + 6] ^= tableLine[0xff & input[iPos + 6]];
-          output[oPos + 7] ^= tableLine[0xff & input[iPos + 7]];
-        }
-
-        /**
-         * For the left bytes, do it one by one.
-         */
-        for (i = extra; i < dataLen; i++, iPos++, oPos++) {
-          output[oPos] ^= tableLine[0xff & input[iPos]];
-        }
-      }
-    }
-  }
-
-  /**
-   * See above. Try to use the byte[] version when possible.
-   */
-  public static void encodeData(byte[] gfTables, ByteBuffer[] inputs,
-                                ByteBuffer[] outputs) {
-    int numInputs = inputs.length;
-    int numOutputs = outputs.length;
-    int dataLen = inputs[0].remaining();
-    int l, i, j, iPos, oPos;
-    ByteBuffer input, output;
-    byte s;
-    final int times = dataLen / 8;
-    final int extra = dataLen - dataLen % 8;
-    byte[] tableLine;
-
-    for (l = 0; l < numOutputs; l++) {
-      output = outputs[l];
-
-      for (j = 0; j < numInputs; j++) {
-        input = inputs[j];
-        iPos = input.position();
-        oPos = output.position();
-
-        s = gfTables[j * 32 + l * numInputs * 32 + 1];
-        tableLine = GF256.gfMulTab()[s & 0xff];
-
-        for (i = 0; i < times; i++, iPos += 8, oPos += 8) {
-          output.put(oPos + 0, (byte) (output.get(oPos + 0) ^
-              tableLine[0xff & input.get(iPos + 0)]));
-          output.put(oPos + 1, (byte) (output.get(oPos + 1) ^
-              tableLine[0xff & input.get(iPos + 1)]));
-          output.put(oPos + 2, (byte) (output.get(oPos + 2) ^
-              tableLine[0xff & input.get(iPos + 2)]));
-          output.put(oPos + 3, (byte) (output.get(oPos + 3) ^
-              tableLine[0xff & input.get(iPos + 3)]));
-          output.put(oPos + 4, (byte) (output.get(oPos + 4) ^
-              tableLine[0xff & input.get(iPos + 4)]));
-          output.put(oPos + 5, (byte) (output.get(oPos + 5) ^
-              tableLine[0xff & input.get(iPos + 5)]));
-          output.put(oPos + 6, (byte) (output.get(oPos + 6) ^
-              tableLine[0xff & input.get(iPos + 6)]));
-          output.put(oPos + 7, (byte) (output.get(oPos + 7) ^
-              tableLine[0xff & input.get(iPos + 7)]));
-        }
-
-        for (i = extra; i < dataLen; i++, iPos++, oPos++) {
-          output.put(oPos, (byte) (output.get(oPos) ^
-              tableLine[0xff & input.get(iPos)]));
-        }
-      }
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/19e8f076/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/coder/TestHHXORErasureCoder.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/coder/TestHHXORErasureCoder.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/coder/TestHHXORErasureCoder.java
index 6372d48..ad346e0 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/coder/TestHHXORErasureCoder.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/coder/TestHHXORErasureCoder.java
@@ -19,7 +19,7 @@ package org.apache.hadoop.io.erasurecode.coder;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.CommonConfigurationKeys;
-import org.apache.hadoop.io.erasurecode.rawcoder.RSRawErasureCoderFactoryLegacy;
+import org.apache.hadoop.io.erasurecode.rawcoder.RSRawErasureCoderFactory;
 import org.junit.Before;
 import org.junit.Test;
 
@@ -51,7 +51,7 @@ public class TestHHXORErasureCoder extends TestHHErasureCoderBase {
      */
     Configuration conf = new Configuration();
     conf.set(CommonConfigurationKeys.IO_ERASURECODE_CODEC_RS_RAWCODER_KEY,
-        RSRawErasureCoderFactoryLegacy.class.getCanonicalName());
+        RSRawErasureCoderFactory.class.getCanonicalName());
     prepare(conf, 10, 4, new int[]{0}, new int[0]);
 
     testCoding(true);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/19e8f076/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/coder/TestRSErasureCoder.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/coder/TestRSErasureCoder.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/coder/TestRSErasureCoder.java
index 213eee6..ee2348e 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/coder/TestRSErasureCoder.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/coder/TestRSErasureCoder.java
@@ -19,7 +19,7 @@ package org.apache.hadoop.io.erasurecode.coder;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.CommonConfigurationKeys;
-import org.apache.hadoop.io.erasurecode.rawcoder.RSRawErasureCoderFactoryLegacy;
+import org.apache.hadoop.io.erasurecode.rawcoder.RSRawErasureCoderFactory;
 import org.junit.Before;
 import org.junit.Rule;
 import org.junit.Test;
@@ -58,20 +58,20 @@ public class TestRSErasureCoder extends TestErasureCoderBase {
      */
     Configuration conf = new Configuration();
     conf.set(CommonConfigurationKeys.IO_ERASURECODE_CODEC_RS_RAWCODER_KEY,
-        RSRawErasureCoderFactoryLegacy.class.getCanonicalName());
+        RSRawErasureCoderFactory.class.getCanonicalName());
     prepare(conf, 10, 4, new int[]{0}, new int[0]);
 
     testCoding(true);
     testCoding(true);
   }
-  
+
   @Test
   public void testCodingDirectBuffer_10x4_erasing_p1() {
     prepare(null, 10, 4, new int[]{}, new int[]{1});
     testCoding(true);
     testCoding(true);
   }
-  
+
   @Test
   public void testCodingDirectBuffer_10x4_erasing_d2() {
     prepare(null, 10, 4, new int[] {2}, new int[] {});

http://git-wip-us.apache.org/repos/asf/hadoop/blob/19e8f076/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestRSRawCoder.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestRSRawCoder.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestRSRawCoder.java
new file mode 100644
index 0000000..5216b9b
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestRSRawCoder.java
@@ -0,0 +1,33 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.io.erasurecode.rawcoder;
+
+import org.junit.Before;
+
+/**
+ * Test the new raw Reed-solomon coder implemented in Java.
+ */
+public class TestRSRawCoder extends TestRSRawCoderBase {
+
+  @Before
+  public void setup() {
+    this.encoderClass = RSRawEncoder.class;
+    this.decoderClass = RSRawDecoder.class;
+    setAllowDump(false);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/19e8f076/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestRSRawCoder2.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestRSRawCoder2.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestRSRawCoder2.java
deleted file mode 100644
index 3e11d14..0000000
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestRSRawCoder2.java
+++ /dev/null
@@ -1,33 +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.io.erasurecode.rawcoder;
-
-import org.junit.Before;
-
-/**
- * Test the new raw Reed-solomon coder implemented in Java.
- */
-public class TestRSRawCoder2 extends TestRSRawCoderBase {
-
-  @Before
-  public void setup() {
-    this.encoderClass = RSRawEncoder2.class;
-    this.decoderClass = RSRawDecoder2.class;
-    setAllowDump(false);
-  }
-}


[16/46] hadoop git commit: YARN-4816. Fix incompatible change in SystemClock.

Posted by ae...@apache.org.
YARN-4816. Fix incompatible change in SystemClock.


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

Branch: refs/heads/HDFS-7240
Commit: eba66a64d28b50a660d6f537c767677f5fa0f7ea
Parents: 19e8f07
Author: Siddharth Seth <ss...@apache.org>
Authored: Mon Mar 14 20:14:33 2016 -0700
Committer: Siddharth Seth <ss...@apache.org>
Committed: Mon Mar 14 20:14:33 2016 -0700

----------------------------------------------------------------------
 .../src/main/java/org/apache/hadoop/yarn/util/SystemClock.java  | 5 +++--
 1 file changed, 3 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/eba66a64/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/SystemClock.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/SystemClock.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/SystemClock.java
index c60368f..72e42d1 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/SystemClock.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/SystemClock.java
@@ -38,11 +38,12 @@ public final class SystemClock implements Clock {
     return INSTANCE;
   }
 
-  private SystemClock() {
+  @Deprecated
+  public SystemClock() {
     // do nothing
   }
 
   public long getTime() {
     return System.currentTimeMillis();
   }
-}
\ No newline at end of file
+}


[03/46] hadoop git commit: Revert "MAPREDUCE-6520. Migrate MR Client test cases part 1."

Posted by ae...@apache.org.
Revert "MAPREDUCE-6520. Migrate MR Client test cases part 1."

This reverts commit ef68b441957c6deb65adae2272d60ac929d00733.


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

Branch: refs/heads/HDFS-7240
Commit: 6876b9f9f990553565eb11aa17635c3874864d7f
Parents: ef68b44
Author: Akira Ajisaka <aa...@apache.org>
Authored: Fri Mar 11 22:50:57 2016 +0900
Committer: Akira Ajisaka <aa...@apache.org>
Committed: Fri Mar 11 22:50:57 2016 +0900

----------------------------------------------------------------------
 .../hadoop/conf/TestNoDefaultsJobConf.java      | 30 ++++-----------
 .../apache/hadoop/mapred/HadoopTestCase.java    | 17 +++++----
 .../hadoop/mapred/NotificationTestCase.java     | 15 ++------
 .../hadoop/mapred/TestFileOutputFormat.java     |  4 --
 .../apache/hadoop/mapred/TestTaskCommit.java    | 31 +++++----------
 .../mapred/jobcontrol/TestLocalJobControl.java  |  3 --
 .../hadoop/mapred/lib/TestChainMapReduce.java   | 20 +---------
 .../mapred/lib/TestKeyFieldBasedComparator.java |  2 -
 .../hadoop/mapred/lib/TestMultipleInputs.java   | 17 ++++-----
 .../hadoop/mapred/lib/TestMultipleOutputs.java  | 26 +------------
 .../mapred/lib/TestMultithreadedMapRunner.java  | 21 +---------
 .../org/apache/hadoop/mapreduce/TestChild.java  | 10 +----
 .../hadoop/mapreduce/TestNoJobSetupCleanup.java |  6 +--
 .../hadoop/mapreduce/TestTaskContext.java       |  2 -
 .../mapreduce/lib/chain/TestChainErrors.java    |  9 -----
 .../mapreduce/lib/chain/TestMapReduceChain.java |  6 ---
 .../lib/chain/TestSingleElementChain.java       |  4 --
 .../lib/db/TestDataDrivenDBInputFormat.java     | 40 ++++++--------------
 .../mapreduce/lib/input/TestMultipleInputs.java |  6 +--
 .../lib/jobcontrol/TestMapReduceJobControl.java |  8 +---
 .../lib/map/TestMultithreadedMapper.java        | 12 +-----
 .../lib/output/TestJobOutputCommitter.java      | 16 ++------
 .../lib/output/TestMRMultipleOutputs.java       | 17 +--------
 .../TestMRKeyFieldBasedComparator.java          | 20 +++-------
 .../hadoop/examples/terasort/TestTeraSort.java  | 11 +-----
 25 files changed, 68 insertions(+), 285 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/6876b9f9/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/conf/TestNoDefaultsJobConf.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/conf/TestNoDefaultsJobConf.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/conf/TestNoDefaultsJobConf.java
index e2d75ab..b47d5ef 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/conf/TestNoDefaultsJobConf.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/conf/TestNoDefaultsJobConf.java
@@ -17,30 +17,17 @@
  */
 package org.apache.hadoop.conf;
 
-import org.apache.hadoop.fs.FileUtil;
+import org.junit.Assert;
+
+import org.apache.hadoop.mapred.*;
+import org.apache.hadoop.mapreduce.MRConfig;
+import org.apache.hadoop.mapreduce.server.jobtracker.JTConfig;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.io.LongWritable;
 import org.apache.hadoop.io.Text;
-import org.apache.hadoop.mapred.FileInputFormat;
-import org.apache.hadoop.mapred.FileOutputFormat;
-import org.apache.hadoop.mapred.HadoopTestCase;
-import org.apache.hadoop.mapred.JobClient;
-import org.apache.hadoop.mapred.JobConf;
-import org.apache.hadoop.mapred.TextInputFormat;
-import org.apache.hadoop.mapred.TextOutputFormat;
-import org.apache.hadoop.mapred.Utils;
-import org.junit.Test;
-
-import java.io.BufferedReader;
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.InputStreamReader;
-import java.io.OutputStream;
-import java.io.OutputStreamWriter;
-import java.io.Writer;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
+
+import java.io.*;
 
 /**
  * This testcase tests that a JobConf without default values submits jobs
@@ -53,7 +40,6 @@ public class TestNoDefaultsJobConf extends HadoopTestCase {
     super(HadoopTestCase.CLUSTER_MR, HadoopTestCase.DFS_FS, 1, 1);
   }
 
-  @Test
   public void testNoDefaults() throws Exception {
     JobConf configuration = new JobConf();
     assertTrue(configuration.get("hadoop.tmp.dir", null) != null);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6876b9f9/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/HadoopTestCase.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/HadoopTestCase.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/HadoopTestCase.java
index 277c0fd..3cd0668 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/HadoopTestCase.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/HadoopTestCase.java
@@ -18,12 +18,13 @@
 
 package org.apache.hadoop.mapred;
 
-import org.apache.hadoop.fs.FileSystem;
+import junit.framework.TestCase;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.mapreduce.MRConfig;
-import org.junit.After;
-import org.junit.Before;
 
+import java.io.File;
 import java.io.IOException;
 
 /**
@@ -37,7 +38,7 @@ import java.io.IOException;
  * Job Configurations should be created using a configuration returned by the
  * 'createJobConf()' method.
  */
-public abstract class HadoopTestCase {
+public abstract class HadoopTestCase extends TestCase {
   public static final int LOCAL_MR = 1;
   public static final int CLUSTER_MR = 2;
   public static final int LOCAL_FS = 4;
@@ -139,8 +140,8 @@ public abstract class HadoopTestCase {
    *
    * @throws Exception
    */
-  @Before
-  public void setUp() throws Exception {
+  protected void setUp() throws Exception {
+    super.setUp();
     if (localFS) {
       fileSystem = FileSystem.getLocal(new JobConf());
     }
@@ -163,8 +164,7 @@ public abstract class HadoopTestCase {
    *
    * @throws Exception
    */
-  @After
-  public void tearDown() throws Exception {
+  protected void tearDown() throws Exception {
     try {
       if (mrCluster != null) {
         mrCluster.shutdown();
@@ -181,6 +181,7 @@ public abstract class HadoopTestCase {
     catch (Exception ex) {
       System.out.println(ex);
     }
+    super.tearDown();
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6876b9f9/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/NotificationTestCase.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/NotificationTestCase.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/NotificationTestCase.java
index 1f657cf..d2ea74e 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/NotificationTestCase.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/NotificationTestCase.java
@@ -34,13 +34,6 @@ import javax.servlet.ServletException;
 import java.io.IOException;
 import java.io.DataOutputStream;
 
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.assertEquals;
-import org.junit.Before;
-import org.junit.After;
-import org.junit.Test;
-
-
 /**
  * Base class to test Job end notification in local and cluster mode.
  *
@@ -147,19 +140,17 @@ public abstract class NotificationTestCase extends HadoopTestCase {
     return conf;
   }
 
-  @Before
-  public void setUp() throws Exception {
+
+  protected void setUp() throws Exception {
     super.setUp();
     startHttpServer();
   }
 
-  @After
-  public void tearDown() throws Exception {
+  protected void tearDown() throws Exception {
     stopHttpServer();
     super.tearDown();
   }
 
-  @Test
   public void testMR() throws Exception {
 
     System.out.println(launchWordCount(this.createJobConf(),

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6876b9f9/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestFileOutputFormat.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestFileOutputFormat.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestFileOutputFormat.java
index 3141235..81b53cc 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestFileOutputFormat.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestFileOutputFormat.java
@@ -30,16 +30,12 @@ import java.io.IOException;
 import java.io.OutputStream;
 import java.util.Iterator;
 
-import org.junit.Test;
-import static org.junit.Assert.assertTrue;
-
 public class TestFileOutputFormat extends HadoopTestCase {
 
   public TestFileOutputFormat() throws IOException {
     super(HadoopTestCase.LOCAL_MR, HadoopTestCase.LOCAL_FS, 1, 1);
   }
 
-  @Test
   public void testCustomFile() throws Exception {
     Path inDir = new Path("testing/fileoutputformat/input");
     Path outDir = new Path("testing/fileoutputformat/output");

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6876b9f9/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestTaskCommit.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestTaskCommit.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestTaskCommit.java
index bed545e..bad06e9 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestTaskCommit.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestTaskCommit.java
@@ -17,6 +17,9 @@
  */
 package org.apache.hadoop.mapred;
 
+import java.io.File;
+import java.io.IOException;
+
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.Path;
@@ -24,18 +27,9 @@ import org.apache.hadoop.io.Text;
 import org.apache.hadoop.ipc.ProtocolSignature;
 import org.apache.hadoop.mapred.SortedRanges.Range;
 import org.apache.hadoop.mapreduce.TaskType;
+import org.apache.hadoop.mapreduce.checkpoint.CheckpointID;
+import org.apache.hadoop.mapreduce.checkpoint.FSCheckpointID;
 import org.apache.hadoop.mapreduce.checkpoint.TaskCheckpointID;
-import org.junit.After;
-import org.junit.Test;
-
-import java.io.File;
-import java.io.IOException;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-
 
 
 public class TestTaskCommit extends HadoopTestCase {
@@ -86,13 +80,12 @@ public class TestTaskCommit extends HadoopTestCase {
     super(LOCAL_MR, LOCAL_FS, 1, 1);
   }
   
-  @After
+  @Override
   public void tearDown() throws Exception {
     super.tearDown();
     FileUtil.fullyDelete(new File(rootDir.toString()));
   }
-
-  @Test
+  
   public void testCommitFail() throws IOException {
     final Path inDir = new Path(rootDir, "./input");
     final Path outDir = new Path(rootDir, "./output");
@@ -206,7 +199,6 @@ public class TestTaskCommit extends HadoopTestCase {
    * 
    * @throws Exception
    */
-  @Test
   public void testTaskCleanupDoesNotCommit() throws Exception {
     // Mimic a job with a special committer that does not cleanup
     // files when a task fails.
@@ -253,27 +245,23 @@ public class TestTaskCommit extends HadoopTestCase {
     assertTrue("Task did not succeed", umbilical.taskDone);
   }
 
-  @Test
   public void testCommitRequiredForMapTask() throws Exception {
     Task testTask = createDummyTask(TaskType.MAP);
     assertTrue("MapTask should need commit", testTask.isCommitRequired());
   }
 
-  @Test
   public void testCommitRequiredForReduceTask() throws Exception {
     Task testTask = createDummyTask(TaskType.REDUCE);
     assertTrue("ReduceTask should need commit", testTask.isCommitRequired());
   }
-
-  @Test
+  
   public void testCommitNotRequiredForJobSetup() throws Exception {
     Task testTask = createDummyTask(TaskType.MAP);
     testTask.setJobSetupTask();
     assertFalse("Job setup task should not need commit", 
         testTask.isCommitRequired());
   }
-
-  @Test
+  
   public void testCommitNotRequiredForJobCleanup() throws Exception {
     Task testTask = createDummyTask(TaskType.MAP);
     testTask.setJobCleanupTask();
@@ -281,7 +269,6 @@ public class TestTaskCommit extends HadoopTestCase {
         testTask.isCommitRequired());
   }
 
-  @Test
   public void testCommitNotRequiredForTaskCleanup() throws Exception {
     Task testTask = createDummyTask(TaskType.REDUCE);
     testTask.setTaskCleanupTask();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6876b9f9/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/jobcontrol/TestLocalJobControl.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/jobcontrol/TestLocalJobControl.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/jobcontrol/TestLocalJobControl.java
index 07b1306..8d35dcf 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/jobcontrol/TestLocalJobControl.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/jobcontrol/TestLocalJobControl.java
@@ -27,8 +27,6 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.mapred.HadoopTestCase;
 import org.apache.hadoop.mapred.JobConf;
-import org.junit.Test;
-import static org.junit.Assert.assertEquals;
 
 /**
  * HadoopTestCase that tests the local job runner.
@@ -61,7 +59,6 @@ public class TestLocalJobControl extends HadoopTestCase {
    * object. Finally, it creates a thread to run the JobControl object and
    * monitors/reports the job states.
    */
-  @Test
   public void testLocalJobControlDataCopy() throws Exception {
 
     FileSystem fs = FileSystem.get(createJobConf());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6876b9f9/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/lib/TestChainMapReduce.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/lib/TestChainMapReduce.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/lib/TestChainMapReduce.java
index 0933ece..37cb91f 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/lib/TestChainMapReduce.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/lib/TestChainMapReduce.java
@@ -21,29 +21,12 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.LongWritable;
 import org.apache.hadoop.io.Text;
-import org.apache.hadoop.mapred.FileInputFormat;
-import org.apache.hadoop.mapred.FileOutputFormat;
-import org.apache.hadoop.mapred.HadoopTestCase;
-import org.apache.hadoop.mapred.JobClient;
-import org.apache.hadoop.mapred.JobConf;
-import org.apache.hadoop.mapred.Mapper;
-import org.apache.hadoop.mapred.OutputCollector;
-import org.apache.hadoop.mapred.Reducer;
-import org.apache.hadoop.mapred.Reporter;
-import org.apache.hadoop.mapred.RunningJob;
-import org.apache.hadoop.mapred.TextInputFormat;
-import org.apache.hadoop.mapred.TextOutputFormat;
-import org.junit.Test;
+import org.apache.hadoop.mapred.*;
 
 import java.io.DataOutputStream;
 import java.io.IOException;
 import java.util.Iterator;
 
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNotSame;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-
 public class TestChainMapReduce extends HadoopTestCase {
 
   private static Path getFlagDir(boolean local) {
@@ -84,7 +67,6 @@ public class TestChainMapReduce extends HadoopTestCase {
     super(HadoopTestCase.LOCAL_MR, HadoopTestCase.LOCAL_FS, 1, 1);
   }
 
-  @Test
   public void testChain() throws Exception {
     Path inDir = new Path("testing/chain/input");
     Path outDir = new Path("testing/chain/output");

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6876b9f9/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/lib/TestKeyFieldBasedComparator.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/lib/TestKeyFieldBasedComparator.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/lib/TestKeyFieldBasedComparator.java
index 35b3f24..34a4d2c 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/lib/TestKeyFieldBasedComparator.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/lib/TestKeyFieldBasedComparator.java
@@ -36,8 +36,6 @@ import org.apache.hadoop.mapred.TextOutputFormat;
 import org.apache.hadoop.mapred.Utils;
 import org.junit.After;
 import org.junit.Test;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
 
 import java.io.BufferedReader;
 import java.io.File;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6876b9f9/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/lib/TestMultipleInputs.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/lib/TestMultipleInputs.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/lib/TestMultipleInputs.java
index 3a9cb9e..e5c6d75 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/lib/TestMultipleInputs.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/lib/TestMultipleInputs.java
@@ -17,6 +17,11 @@
  */
 package org.apache.hadoop.mapred.lib;
 
+import java.io.IOException;
+import java.util.Map;
+
+import junit.framework.TestCase;
+
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.mapred.InputFormat;
 import org.apache.hadoop.mapred.JobConf;
@@ -25,19 +30,12 @@ import org.apache.hadoop.mapred.Mapper;
 import org.apache.hadoop.mapred.OutputCollector;
 import org.apache.hadoop.mapred.Reporter;
 import org.apache.hadoop.mapred.TextInputFormat;
-import org.junit.Test;
-
-import java.io.IOException;
-import java.util.Map;
-
-import static org.junit.Assert.assertEquals;
 
 /**
  * @see TestDelegatingInputFormat
  */
-public class TestMultipleInputs {
-
-  @Test
+public class TestMultipleInputs extends TestCase {
+  
   public void testAddInputPathWithFormat() {
     final JobConf conf = new JobConf();
     MultipleInputs.addInputPath(conf, new Path("/foo"), TextInputFormat.class);
@@ -50,7 +48,6 @@ public class TestMultipleInputs {
        .getClass());
   }
 
-  @Test
   public void testAddInputPathWithMapper() {
     final JobConf conf = new JobConf();
     MultipleInputs.addInputPath(conf, new Path("/foo"), TextInputFormat.class,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6876b9f9/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/lib/TestMultipleOutputs.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/lib/TestMultipleOutputs.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/lib/TestMultipleOutputs.java
index f3e5893..59c0a97 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/lib/TestMultipleOutputs.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/lib/TestMultipleOutputs.java
@@ -24,23 +24,7 @@ import org.apache.hadoop.io.LongWritable;
 import org.apache.hadoop.io.SequenceFile;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.serializer.JavaSerializationComparator;
-import org.apache.hadoop.mapred.Counters;
-import org.apache.hadoop.mapred.FileInputFormat;
-import org.apache.hadoop.mapred.FileOutputFormat;
-import org.apache.hadoop.mapred.HadoopTestCase;
-import org.apache.hadoop.mapred.JobClient;
-import org.apache.hadoop.mapred.JobConf;
-import org.apache.hadoop.mapred.Mapper;
-import org.apache.hadoop.mapred.OutputCollector;
-import org.apache.hadoop.mapred.Reducer;
-import org.apache.hadoop.mapred.Reporter;
-import org.apache.hadoop.mapred.RunningJob;
-import org.apache.hadoop.mapred.SequenceFileOutputFormat;
-import org.apache.hadoop.mapred.TextInputFormat;
-import org.apache.hadoop.mapred.TextOutputFormat;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Test;
+import org.apache.hadoop.mapred.*;
 
 import java.io.BufferedReader;
 import java.io.DataOutputStream;
@@ -48,23 +32,17 @@ import java.io.IOException;
 import java.io.InputStreamReader;
 import java.util.Iterator;
 
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-
 public class TestMultipleOutputs extends HadoopTestCase {
 
   public TestMultipleOutputs() throws IOException {
     super(HadoopTestCase.LOCAL_MR, HadoopTestCase.LOCAL_FS, 1, 1);
   }
 
-  @Test
   public void testWithoutCounters() throws Exception {
     _testMultipleOutputs(false);
     _testMOWithJavaSerialization(false);
   }
 
-  @Test
   public void testWithCounters() throws Exception {
     _testMultipleOutputs(true);
     _testMOWithJavaSerialization(true);
@@ -84,7 +62,6 @@ public class TestMultipleOutputs extends HadoopTestCase {
     return dir;
   }
 
-  @Before
   public void setUp() throws Exception {
     super.setUp();
     Path rootDir = getDir(ROOT_DIR);
@@ -98,7 +75,6 @@ public class TestMultipleOutputs extends HadoopTestCase {
     }
   }
 
-  @After
   public void tearDown() throws Exception {
     Path rootDir = getDir(ROOT_DIR);
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6876b9f9/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/lib/TestMultithreadedMapRunner.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/lib/TestMultithreadedMapRunner.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/lib/TestMultithreadedMapRunner.java
index 1059d29..7e224cd 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/lib/TestMultithreadedMapRunner.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/lib/TestMultithreadedMapRunner.java
@@ -22,45 +22,26 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.LongWritable;
 import org.apache.hadoop.io.Text;
-import org.apache.hadoop.mapred.FileInputFormat;
-import org.apache.hadoop.mapred.FileOutputFormat;
-import org.apache.hadoop.mapred.HadoopTestCase;
-import org.apache.hadoop.mapred.JobClient;
-import org.apache.hadoop.mapred.JobConf;
-import org.apache.hadoop.mapred.Mapper;
-import org.apache.hadoop.mapred.OutputCollector;
-import org.apache.hadoop.mapred.Reducer;
-import org.apache.hadoop.mapred.Reporter;
-import org.apache.hadoop.mapred.RunningJob;
-import org.apache.hadoop.mapred.TextInputFormat;
-import org.apache.hadoop.mapred.TextOutputFormat;
+import org.apache.hadoop.mapred.*;
 import org.apache.hadoop.mapreduce.lib.map.MultithreadedMapper;
-import org.junit.Test;
 
 import java.io.DataOutputStream;
 import java.io.IOException;
 import java.util.Iterator;
 
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-
 public class TestMultithreadedMapRunner extends HadoopTestCase {
 
   public TestMultithreadedMapRunner() throws IOException {
     super(HadoopTestCase.LOCAL_MR, HadoopTestCase.LOCAL_FS, 1, 1);
   }
 
-  @Test
   public void testOKRun() throws Exception {
     run(false, false);
   }
 
-  @Test
   public void testIOExRun() throws Exception {
     run(true, false);
   }
-
-  @Test
   public void testRuntimeExRun() throws Exception {
     run(false, true);
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6876b9f9/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestChild.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestChild.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestChild.java
index 338f117..d5afe63 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestChild.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestChild.java
@@ -30,13 +30,6 @@ import org.apache.hadoop.io.Text;
 import org.apache.hadoop.mapred.HadoopTestCase;
 import org.apache.hadoop.mapred.JobConf;
 import org.apache.log4j.Level;
-import org.junit.Before;
-import org.junit.Test;
-
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
 
 public class TestChild extends HadoopTestCase {
   private static String TEST_ROOT_DIR =
@@ -152,8 +145,7 @@ public class TestChild extends HadoopTestCase {
         + numPartFiles, list.length == numPartFiles);
     return job;
   }
-
-  @Test
+  
   public void testChild() throws Exception {
     try {
       submitAndValidateJob(createJobConf(), 1, 1, true);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6876b9f9/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestNoJobSetupCleanup.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestNoJobSetupCleanup.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestNoJobSetupCleanup.java
index 7520f38..5d36c92 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestNoJobSetupCleanup.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestNoJobSetupCleanup.java
@@ -30,9 +30,6 @@ import org.apache.hadoop.mapred.HadoopTestCase;
 import org.apache.hadoop.mapreduce.lib.output.TextOutputFormat;
 import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl;
 import org.junit.Ignore;
-import org.junit.Test;
-import static org.junit.Assert.assertTrue;
-
 @Ignore
 public class TestNoJobSetupCleanup extends HadoopTestCase {
   private static String TEST_ROOT_DIR =
@@ -71,8 +68,7 @@ public class TestNoJobSetupCleanup extends HadoopTestCase {
         + numPartFiles, list.length == numPartFiles);
     return job;
   }
-
-  @Test
+  
   public void testNoJobSetupCleanup() throws Exception {
     try {
       Configuration conf = createJobConf();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6876b9f9/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestTaskContext.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestTaskContext.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestTaskContext.java
index 67daaa4..bf742c4 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestTaskContext.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestTaskContext.java
@@ -33,8 +33,6 @@ import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.Ignore;
 import org.junit.Test;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
 
 /**
  * Tests context api and {@link StatusReporter#getProgress()} via 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6876b9f9/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/chain/TestChainErrors.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/chain/TestChainErrors.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/chain/TestChainErrors.java
index 46024bc..2dfcf41 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/chain/TestChainErrors.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/chain/TestChainErrors.java
@@ -29,10 +29,6 @@ import org.apache.hadoop.mapreduce.Job;
 import org.apache.hadoop.mapreduce.MapReduceTestUtil;
 import org.apache.hadoop.mapreduce.Mapper;
 import org.apache.hadoop.mapreduce.Reducer;
-import org.junit.Test;
-
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.assertEquals;
 
 /**
  * Tests error conditions in ChainMapper/ChainReducer.
@@ -55,7 +51,6 @@ public class TestChainErrors extends HadoopTestCase {
    * 
    * @throws Exception
    */
-  @Test
   public void testChainSubmission() throws Exception {
 
     Configuration conf = createJobConf();
@@ -94,7 +89,6 @@ public class TestChainErrors extends HadoopTestCase {
    * 
    * @throws Exception
    */
-  @Test
   public void testChainFail() throws Exception {
 
     Configuration conf = createJobConf();
@@ -120,7 +114,6 @@ public class TestChainErrors extends HadoopTestCase {
    * 
    * @throws Exception
    */
-  @Test
   public void testReducerFail() throws Exception {
 
     Configuration conf = createJobConf();
@@ -146,7 +139,6 @@ public class TestChainErrors extends HadoopTestCase {
    * 
    * @throws Exception
    */
-  @Test
   public void testChainMapNoOuptut() throws Exception {
     Configuration conf = createJobConf();
     String expectedOutput = "";
@@ -171,7 +163,6 @@ public class TestChainErrors extends HadoopTestCase {
    * 
    * @throws Exception
    */
-  @Test
   public void testChainReduceNoOuptut() throws Exception {
     Configuration conf = createJobConf();
     String expectedOutput = "";

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6876b9f9/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/chain/TestMapReduceChain.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/chain/TestMapReduceChain.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/chain/TestMapReduceChain.java
index aaaaf51..971ea68 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/chain/TestMapReduceChain.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/chain/TestMapReduceChain.java
@@ -30,11 +30,6 @@ import org.apache.hadoop.mapreduce.Job;
 import org.apache.hadoop.mapreduce.MapReduceTestUtil;
 import org.apache.hadoop.mapreduce.Mapper;
 import org.apache.hadoop.mapreduce.Reducer;
-import org.junit.Test;
-
-import static org.junit.Assert.fail;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
 
 public class TestMapReduceChain extends HadoopTestCase {
 
@@ -68,7 +63,6 @@ public class TestMapReduceChain extends HadoopTestCase {
     super(HadoopTestCase.LOCAL_MR, HadoopTestCase.LOCAL_FS, 1, 1);
   }
 
-  @Test
   public void testChain() throws Exception {
     Path inDir = new Path(localPathRoot, "testing/chain/input");
     Path outDir = new Path(localPathRoot, "testing/chain/output");

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6876b9f9/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/chain/TestSingleElementChain.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/chain/TestSingleElementChain.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/chain/TestSingleElementChain.java
index f78ac70..06cfe1c 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/chain/TestSingleElementChain.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/chain/TestSingleElementChain.java
@@ -26,9 +26,6 @@ import org.apache.hadoop.mapreduce.Job;
 import org.apache.hadoop.mapreduce.MapReduceTestUtil;
 import org.apache.hadoop.mapreduce.lib.map.TokenCounterMapper;
 import org.apache.hadoop.mapreduce.lib.reduce.IntSumReducer;
-import org.junit.Test;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
 
 import java.io.IOException;
 
@@ -45,7 +42,6 @@ public class TestSingleElementChain extends HadoopTestCase {
   }
 
   // test chain mapper and reducer by adding single mapper and reducer to chain
-  @Test
   public void testNoChain() throws Exception {
     Path inDir = new Path(localPathRoot, "testing/chain/input");
     Path outDir = new Path(localPathRoot, "testing/chain/output");

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6876b9f9/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/db/TestDataDrivenDBInputFormat.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/db/TestDataDrivenDBInputFormat.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/db/TestDataDrivenDBInputFormat.java
index 81a3249..37f9364 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/db/TestDataDrivenDBInputFormat.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/db/TestDataDrivenDBInputFormat.java
@@ -18,40 +18,25 @@
 
 package org.apache.hadoop.mapreduce.lib.db;
 
+import java.sql.*;
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
+//import org.apache.hadoop.examples.DBCountPageView;
+import org.apache.hadoop.fs.*;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.NullWritable;
 import org.apache.hadoop.io.WritableComparable;
 import org.apache.hadoop.mapred.HadoopTestCase;
-import org.apache.hadoop.mapreduce.Job;
-import org.apache.hadoop.mapreduce.Mapper;
-import org.apache.hadoop.mapreduce.Reducer;
-import org.apache.hadoop.mapreduce.TaskCounter;
-import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
+import org.apache.hadoop.mapreduce.*;
+import org.apache.hadoop.mapreduce.lib.db.*;
+import org.apache.hadoop.mapreduce.lib.input.*;
+import org.apache.hadoop.mapreduce.lib.output.*;
 import org.apache.hadoop.util.StringUtils;
 import org.hsqldb.server.Server;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Test;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-import java.sql.Connection;
-import java.sql.Date;
-import java.sql.DriverManager;
-import java.sql.PreparedStatement;
-import java.sql.ResultSet;
-import java.sql.SQLException;
-import java.sql.Statement;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-
-//import org.apache.hadoop.examples.DBCountPageView;
 
 /**
  * Test aspects of DataDrivenDBInputFormat
@@ -124,13 +109,11 @@ public class TestDataDrivenDBInputFormat extends HadoopTestCase {
     createConnection(driverClassName, url);
   }
 
-  @Before
   public void setUp() throws Exception {
     initialize(DRIVER_CLASS, DB_URL);
     super.setUp();
   }
 
-  @After
   public void tearDown() throws Exception {
     super.tearDown();
     shutdown();
@@ -187,7 +170,6 @@ public class TestDataDrivenDBInputFormat extends HadoopTestCase {
     }
   }
 
-  @Test
   public void testDateSplits() throws Exception {
     Statement s = connection.createStatement();
     final String DATE_TABLE = "datetable";

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6876b9f9/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/input/TestMultipleInputs.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/input/TestMultipleInputs.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/input/TestMultipleInputs.java
index 632c40e..c868050 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/input/TestMultipleInputs.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/input/TestMultipleInputs.java
@@ -38,8 +38,6 @@ import org.apache.hadoop.mapreduce.Mapper;
 import org.apache.hadoop.mapreduce.Reducer;
 import org.junit.Before;
 import org.junit.Test;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
 
 /**
  * @see TestDelegatingInputFormat
@@ -141,7 +139,7 @@ public class TestMultipleInputs extends HadoopTestCase {
     assertTrue(output.readLine().equals("e 2"));
   }
 
-  @Test
+  @SuppressWarnings("unchecked")
   public void testAddInputPathWithFormat() throws IOException {
     final Job conf = Job.getInstance();
     MultipleInputs.addInputPath(conf, new Path("/foo"), TextInputFormat.class);
@@ -154,7 +152,7 @@ public class TestMultipleInputs extends HadoopTestCase {
        .getClass());
   }
 
-  @Test
+  @SuppressWarnings("unchecked")
   public void testAddInputPathWithMapper() throws IOException {
     final Job conf = Job.getInstance();
     MultipleInputs.addInputPath(conf, new Path("/foo"), TextInputFormat.class,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6876b9f9/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/jobcontrol/TestMapReduceJobControl.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/jobcontrol/TestMapReduceJobControl.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/jobcontrol/TestMapReduceJobControl.java
index d86ddd0..14c64bd 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/jobcontrol/TestMapReduceJobControl.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/jobcontrol/TestMapReduceJobControl.java
@@ -33,9 +33,6 @@ import org.apache.hadoop.mapred.HadoopTestCase;
 import org.apache.hadoop.mapreduce.Job;
 import org.apache.hadoop.mapreduce.MapReduceTestUtil;
 import org.junit.Test;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
 
 /**
  * This class performs unit test for Job/JobControl classes.
@@ -123,8 +120,7 @@ public class TestMapReduceJobControl extends HadoopTestCase {
       } catch (Exception e) {}
     }
   }
-
-  @Test
+  
   public void testJobControlWithFailJob() throws Exception {
     LOG.info("Starting testJobControlWithFailJob");
     Configuration conf = createJobConf();
@@ -148,7 +144,6 @@ public class TestMapReduceJobControl extends HadoopTestCase {
     theControl.stop();
   }
 
-  @Test
   public void testJobControlWithKillJob() throws Exception {
     LOG.info("Starting testJobControlWithKillJob");
 
@@ -187,7 +182,6 @@ public class TestMapReduceJobControl extends HadoopTestCase {
     theControl.stop();
   }
 
-  @Test
   public void testJobControl() throws Exception {
     LOG.info("Starting testJobControl");
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6876b9f9/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/map/TestMultithreadedMapper.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/map/TestMultithreadedMapper.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/map/TestMultithreadedMapper.java
index 5096192..52b0e70 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/map/TestMultithreadedMapper.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/map/TestMultithreadedMapper.java
@@ -23,33 +23,23 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.LongWritable;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.mapred.HadoopTestCase;
-import org.apache.hadoop.mapreduce.Job;
-import org.apache.hadoop.mapreduce.MapReduceTestUtil;
-import org.apache.hadoop.mapreduce.Mapper;
-import org.apache.hadoop.mapreduce.Reducer;
-import org.junit.Test;
+import org.apache.hadoop.mapreduce.*;
 
 import java.io.IOException;
 
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-
 public class TestMultithreadedMapper extends HadoopTestCase {
 
   public TestMultithreadedMapper() throws IOException {
     super(HadoopTestCase.LOCAL_MR, HadoopTestCase.LOCAL_FS, 1, 1);
   }
 
-  @Test
   public void testOKRun() throws Exception {
     run(false, false);
   }
 
-  @Test
   public void testIOExRun() throws Exception {
     run(true, false);
   }
-  @Test
   public void testRuntimeExRun() throws Exception {
     run(false, true);
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6876b9f9/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/output/TestJobOutputCommitter.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/output/TestJobOutputCommitter.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/output/TestJobOutputCommitter.java
index 19b712f..49b59ca 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/output/TestJobOutputCommitter.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/output/TestJobOutputCommitter.java
@@ -33,11 +33,6 @@ import org.apache.hadoop.mapreduce.MapReduceTestUtil;
 import org.apache.hadoop.mapreduce.OutputCommitter;
 import org.apache.hadoop.mapreduce.OutputFormat;
 import org.apache.hadoop.mapreduce.TaskAttemptContext;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Test;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.assertFalse;
 
 /**
  * A JUnit test to test Map-Reduce job committer.
@@ -59,15 +54,15 @@ public class TestJobOutputCommitter extends HadoopTestCase {
   private FileSystem fs;
   private Configuration conf = null;
 
-  @Before
-  public void setUp() throws Exception {
+  @Override
+  protected void setUp() throws Exception {
     super.setUp();
     conf = createJobConf();
     fs = getFileSystem();
   }
 
-  @After
-  public void tearDown() throws Exception {
+  @Override
+  protected void tearDown() throws Exception {
     fs.delete(new Path(TEST_ROOT_DIR), true);
     super.tearDown();
   }
@@ -224,7 +219,6 @@ public class TestJobOutputCommitter extends HadoopTestCase {
    * 
    * @throws Exception
    */
-  @Test
   public void testDefaultCleanupAndAbort() throws Exception {
     // check with a successful job
     testSuccessfulJob(FileOutputCommitter.SUCCEEDED_FILE_NAME,
@@ -244,7 +238,6 @@ public class TestJobOutputCommitter extends HadoopTestCase {
    * 
    * @throws Exception
    */
-  @Test
   public void testCustomAbort() throws Exception {
     // check with a successful job
     testSuccessfulJob(FileOutputCommitter.SUCCEEDED_FILE_NAME,
@@ -271,7 +264,6 @@ public class TestJobOutputCommitter extends HadoopTestCase {
    * compatibility testing.
    * @throws Exception 
    */
-  @Test
   public void testCustomCleanup() throws Exception {
     // check with a successful job
     testSuccessfulJob(CUSTOM_CLEANUP_FILE_NAME, 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6876b9f9/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/output/TestMRMultipleOutputs.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/output/TestMRMultipleOutputs.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/output/TestMRMultipleOutputs.java
index babd20e..6c432dd 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/output/TestMRMultipleOutputs.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/output/TestMRMultipleOutputs.java
@@ -27,36 +27,23 @@ import org.apache.hadoop.io.SequenceFile;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.serializer.JavaSerializationComparator;
 import org.apache.hadoop.mapred.HadoopTestCase;
-import org.apache.hadoop.mapreduce.CounterGroup;
-import org.apache.hadoop.mapreduce.Job;
-import org.apache.hadoop.mapreduce.MapReduceTestUtil;
-import org.apache.hadoop.mapreduce.Mapper;
-import org.apache.hadoop.mapreduce.Reducer;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Test;
+import org.apache.hadoop.mapreduce.*;
 
 import java.io.BufferedReader;
 import java.io.IOException;
 import java.io.InputStreamReader;
 
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-
 public class TestMRMultipleOutputs extends HadoopTestCase {
 
   public TestMRMultipleOutputs() throws IOException {
     super(HadoopTestCase.LOCAL_MR, HadoopTestCase.LOCAL_FS, 1, 1);
   }
 
-  @Test
   public void testWithoutCounters() throws Exception {
     _testMultipleOutputs(false);
     _testMOWithJavaSerialization(false);
   }
 
-  @Test
   public void testWithCounters() throws Exception {
     _testMultipleOutputs(true);
     _testMOWithJavaSerialization(true);
@@ -70,7 +57,6 @@ public class TestMRMultipleOutputs extends HadoopTestCase {
   private static String TEXT = "text";
   private static String SEQUENCE = "sequence";
 
-  @Before
   public void setUp() throws Exception {
     super.setUp();
     Configuration conf = createJobConf();
@@ -78,7 +64,6 @@ public class TestMRMultipleOutputs extends HadoopTestCase {
     fs.delete(ROOT_DIR, true);
   }
 
-  @After
   public void tearDown() throws Exception {
     Configuration conf = createJobConf();
     FileSystem fs = FileSystem.get(conf);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6876b9f9/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/partition/TestMRKeyFieldBasedComparator.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/partition/TestMRKeyFieldBasedComparator.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/partition/TestMRKeyFieldBasedComparator.java
index 0d75d2f..3a2b831 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/partition/TestMRKeyFieldBasedComparator.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/partition/TestMRKeyFieldBasedComparator.java
@@ -18,6 +18,8 @@
 
 package org.apache.hadoop.mapreduce.lib.partition;
 
+import java.io.*;
+
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.Path;
@@ -30,15 +32,6 @@ import org.apache.hadoop.mapreduce.MRJobConfig;
 import org.apache.hadoop.mapreduce.MapReduceTestUtil;
 import org.apache.hadoop.mapreduce.Reducer;
 import org.apache.hadoop.mapreduce.lib.map.InverseMapper;
-import org.junit.Test;
-
-import java.io.BufferedReader;
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.InputStreamReader;
-
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
 
 
 public class TestMRKeyFieldBasedComparator extends HadoopTestCase {
@@ -52,8 +45,8 @@ public class TestMRKeyFieldBasedComparator extends HadoopTestCase {
     conf = createJobConf();
     conf.set(MRJobConfig.MAP_OUTPUT_KEY_FIELD_SEPERATOR, " ");
   }
-
-  private void testComparator(String keySpec, int expect)
+  
+  private void testComparator(String keySpec, int expect) 
       throws Exception {
     String root = System.getProperty("test.build.data", "/tmp");
     Path inDir = new Path(root, "test_cmp/in");
@@ -100,8 +93,7 @@ public class TestMRKeyFieldBasedComparator extends HadoopTestCase {
       reader.close();
     }
   }
-
-  @Test
+  
   public void testBasicUnixComparator() throws Exception {
     testComparator("-k1,1n", 1);
     testComparator("-k2,2n", 1);
@@ -125,7 +117,7 @@ public class TestMRKeyFieldBasedComparator extends HadoopTestCase {
   byte[] line1_bytes = line1.getBytes();
   byte[] line2_bytes = line2.getBytes();
 
-  private void testWithoutMRJob(String keySpec, int expect) throws Exception {
+  public void testWithoutMRJob(String keySpec, int expect) throws Exception {
     KeyFieldBasedComparator<Void, Void> keyFieldCmp = 
       new KeyFieldBasedComparator<Void, Void>();
     conf.set("mapreduce.partition.keycomparator.options", keySpec);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6876b9f9/hadoop-mapreduce-project/hadoop-mapreduce-examples/src/test/java/org/apache/hadoop/examples/terasort/TestTeraSort.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-examples/src/test/java/org/apache/hadoop/examples/terasort/TestTeraSort.java b/hadoop-mapreduce-project/hadoop-mapreduce-examples/src/test/java/org/apache/hadoop/examples/terasort/TestTeraSort.java
index 391e482..3492089 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-examples/src/test/java/org/apache/hadoop/examples/terasort/TestTeraSort.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-examples/src/test/java/org/apache/hadoop/examples/terasort/TestTeraSort.java
@@ -27,12 +27,6 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.mapred.FileAlreadyExistsException;
 import org.apache.hadoop.mapred.HadoopTestCase;
 import org.apache.hadoop.util.ToolRunner;
-import org.junit.After;
-import org.junit.Test;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.fail;
-
 public class TestTeraSort extends HadoopTestCase {
   private static Log LOG = LogFactory.getLog(TestTeraSort.class);
   
@@ -41,8 +35,7 @@ public class TestTeraSort extends HadoopTestCase {
     super(LOCAL_MR, LOCAL_FS, 1, 1);
   }
 
-  @After
-  public void tearDown() throws Exception {
+  protected void tearDown() throws Exception {
     getFileSystem().delete(new Path(TEST_DIR), true);
     super.tearDown();
   }
@@ -83,7 +76,6 @@ public class TestTeraSort extends HadoopTestCase {
     assertEquals(ToolRunner.run(job, new TeraValidate(), svArgs), 0);
   }
 
-  @Test
   public void testTeraSort() throws Exception {
     // Run TeraGen to generate input for 'terasort'
     runTeraGen(createJobConf(), SORT_INPUT_PATH);
@@ -112,7 +104,6 @@ public class TestTeraSort extends HadoopTestCase {
       TERA_OUTPUT_PATH);
   }
 
-  @Test
   public void testTeraSortWithLessThanTwoArgs() throws Exception {
     String[] args = new String[1];
     assertEquals(new TeraSort().run(args), 2);


[32/46] hadoop git commit: Revert "CapacityScheduler: Improve preemption to only kill containers that would satisfy the incoming request. (Wangda Tan)"

Posted by ae...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/fa7a4352/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 4441c6b..2694957 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
@@ -356,40 +356,4 @@ public class TestUtils {
     CapacityScheduler cs = (CapacityScheduler) rm.getResourceScheduler();
     return cs.getSchedulerApplications().get(appId).getCurrentAppAttempt();
   }
-
-  /**
-   * Get a queue structure:
-   * <pre>
-   *             Root
-   *            /  |  \
-   *           a   b   c
-   *          10   20  70
-   * </pre>
-   */
-  public static Configuration
-  getConfigurationWithMultipleQueues(Configuration config) {
-    CapacitySchedulerConfiguration conf =
-        new CapacitySchedulerConfiguration(config);
-
-    // Define top-level queues
-    conf.setQueues(CapacitySchedulerConfiguration.ROOT,
-        new String[] { "a", "b", "c" });
-
-    final String A = CapacitySchedulerConfiguration.ROOT + ".a";
-    conf.setCapacity(A, 10);
-    conf.setMaximumCapacity(A, 100);
-    conf.setUserLimitFactor(A, 100);
-
-    final String B = CapacitySchedulerConfiguration.ROOT + ".b";
-    conf.setCapacity(B, 20);
-    conf.setMaximumCapacity(B, 100);
-    conf.setUserLimitFactor(B, 100);
-
-    final String C = CapacitySchedulerConfiguration.ROOT + ".c";
-    conf.setCapacity(C, 70);
-    conf.setMaximumCapacity(C, 100);
-    conf.setUserLimitFactor(C, 100);
-
-    return conf;
-  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fa7a4352/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFairSchedulerPreemption.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/fair/TestFairSchedulerPreemption.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFairSchedulerPreemption.java
index 2456594..5bdcc08 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFairSchedulerPreemption.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFairSchedulerPreemption.java
@@ -1451,7 +1451,7 @@ public class TestFairSchedulerPreemption extends FairSchedulerTestBase {
 
     // Trigger container rescheduled event
     scheduler.handle(new ContainerPreemptEvent(appAttemptId, rmContainer,
-            SchedulerEventType.MARK_CONTAINER_FOR_KILLABLE));
+            SchedulerEventType.KILL_PREEMPTED_CONTAINER));
 
     List<ResourceRequest> requests = rmContainer.getResourceRequests();
     // Once recovered, resource request will be present again in app


[45/46] hadoop git commit: YARN-4686. MiniYARNCluster.start() returns before cluster is completely started. Contributed by Eric Badger.

Posted by ae...@apache.org.
YARN-4686. MiniYARNCluster.start() returns before cluster is completely started. Contributed by Eric Badger.


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

Branch: refs/heads/HDFS-7240
Commit: 92b7e0d41302b6b110927f99de5c2b4a4a93c5fd
Parents: 9b623fb
Author: Eric Payne <ep...@apache.org>
Authored: Fri Mar 18 16:11:06 2016 +0000
Committer: Eric Payne <ep...@apache.org>
Committed: Fri Mar 18 16:12:47 2016 +0000

----------------------------------------------------------------------
 .../hadoop/yarn/client/ProtocolHATestBase.java  |  3 +-
 .../hadoop/yarn/client/TestRMFailover.java      |  2 -
 .../yarn/client/api/impl/TestYarnClient.java    | 18 +++++
 .../nodemanager/NodeStatusUpdaterImpl.java      | 50 +++++++------
 .../hadoop/yarn/server/MiniYARNCluster.java     | 75 +++++++-------------
 .../yarn/server/TestMiniYARNClusterForHA.java   |  4 --
 6 files changed, 75 insertions(+), 77 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/92b7e0d4/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/ProtocolHATestBase.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/ProtocolHATestBase.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/ProtocolHATestBase.java
index cf7fcc5..f336b0f 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/ProtocolHATestBase.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/ProtocolHATestBase.java
@@ -217,7 +217,7 @@ public abstract class ProtocolHATestBase extends ClientBaseWithFixes {
   protected void verifyConnections() throws InterruptedException,
       YarnException {
     assertTrue("NMs failed to connect to the RM",
-        cluster.waitForNodeManagersToConnect(20000));
+        cluster.waitForNodeManagersToConnect(5000));
     verifyClientConnection();
   }
 
@@ -279,7 +279,6 @@ public abstract class ProtocolHATestBase extends ClientBaseWithFixes {
     cluster.resetStartFailoverFlag(false);
     cluster.init(conf);
     cluster.start();
-    getAdminService(0).transitionToActive(req);
     assertFalse("RM never turned active", -1 == cluster.getActiveRMIndex());
     verifyConnections();
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/92b7e0d4/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/TestRMFailover.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/TestRMFailover.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/TestRMFailover.java
index cbc220a..f323351 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/TestRMFailover.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/TestRMFailover.java
@@ -142,7 +142,6 @@ public class TestRMFailover extends ClientBaseWithFixes {
     conf.setBoolean(YarnConfiguration.AUTO_FAILOVER_ENABLED, false);
     cluster.init(conf);
     cluster.start();
-    getAdminService(0).transitionToActive(req);
     assertFalse("RM never turned active", -1 == cluster.getActiveRMIndex());
     verifyConnections();
 
@@ -231,7 +230,6 @@ public class TestRMFailover extends ClientBaseWithFixes {
     conf.setBoolean(YarnConfiguration.AUTO_FAILOVER_ENABLED, false);
     cluster.init(conf);
     cluster.start();
-    getAdminService(0).transitionToActive(req);
     assertFalse("RM never turned active", -1 == cluster.getActiveRMIndex());
     verifyConnections();
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/92b7e0d4/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestYarnClient.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestYarnClient.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestYarnClient.java
index 2c34b99..2d11d8a 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestYarnClient.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestYarnClient.java
@@ -1195,6 +1195,24 @@ public class TestYarnClient {
       client.init(yarnConf);
       client.start();
 
+      int attempts;
+      for(attempts = 10; attempts > 0; attempts--) {
+        if (cluster.getResourceManager().getRMContext().getReservationSystem()
+            .getPlan(ReservationSystemTestUtil.reservationQ).getTotalCapacity()
+            .getMemory() > 0) {
+          break;
+        }
+        try {
+          Thread.sleep(100);
+        } catch (InterruptedException e) {
+          e.printStackTrace();
+        }
+      }
+      if (attempts <= 0) {
+        Assert.fail("Exhausted attempts in checking if node capacity was "
+            + "added to the plan");
+      }
+
       // create a reservation
       Clock clock = new UTCClock();
       long arrival = clock.getTime();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/92b7e0d4/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeStatusUpdaterImpl.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/NodeStatusUpdaterImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeStatusUpdaterImpl.java
index 5806731..ad983fe 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeStatusUpdaterImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeStatusUpdaterImpl.java
@@ -96,6 +96,7 @@ public class NodeStatusUpdaterImpl extends AbstractService implements
   private static final Log LOG = LogFactory.getLog(NodeStatusUpdaterImpl.class);
 
   private final Object heartbeatMonitor = new Object();
+  private final Object shutdownMonitor = new Object();
 
   private final Context context;
   private final Dispatcher dispatcher;
@@ -240,15 +241,17 @@ public class NodeStatusUpdaterImpl extends AbstractService implements
   @Override
   protected void serviceStop() throws Exception {
     // the isStopped check is for avoiding multiple unregistrations.
-    if (this.registeredWithRM && !this.isStopped
-        && !isNMUnderSupervisionWithRecoveryEnabled()
-        && !context.getDecommissioned() && !failedToConnect) {
-      unRegisterNM();
+    synchronized(shutdownMonitor) {
+      if (this.registeredWithRM && !this.isStopped
+          && !isNMUnderSupervisionWithRecoveryEnabled()
+          && !context.getDecommissioned() && !failedToConnect) {
+        unRegisterNM();
+      }
+      // Interrupt the updater.
+      this.isStopped = true;
+      stopRMProxy();
+      super.serviceStop();
     }
-    // Interrupt the updater.
-    this.isStopped = true;
-    stopRMProxy();
-    super.serviceStop();
   }
 
   private boolean isNMUnderSupervisionWithRecoveryEnabled() {
@@ -275,19 +278,24 @@ public class NodeStatusUpdaterImpl extends AbstractService implements
 
   protected void rebootNodeStatusUpdaterAndRegisterWithRM() {
     // Interrupt the updater.
-    this.isStopped = true;
-
-    try {
-      statusUpdater.join();
-      registerWithRM();
-      statusUpdater = new Thread(statusUpdaterRunnable, "Node Status Updater");
-      this.isStopped = false;
-      statusUpdater.start();
-      LOG.info("NodeStatusUpdater thread is reRegistered and restarted");
-    } catch (Exception e) {
-      String errorMessage = "Unexpected error rebooting NodeStatusUpdater";
-      LOG.error(errorMessage, e);
-      throw new YarnRuntimeException(e);
+    synchronized(shutdownMonitor) {
+      if(this.isStopped) {
+        LOG.info("Currently being shutdown. Aborting reboot");
+        return;
+      }
+      this.isStopped = true;
+      try {
+        statusUpdater.join();
+        registerWithRM();
+        statusUpdater = new Thread(statusUpdaterRunnable, "Node Status Updater");
+        statusUpdater.start();
+        this.isStopped = false;
+        LOG.info("NodeStatusUpdater thread is reRegistered and restarted");
+      } catch (Exception e) {
+        String errorMessage = "Unexpected error rebooting NodeStatusUpdater";
+        LOG.error(errorMessage, e);
+        throw new YarnRuntimeException(e);
+      }
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/92b7e0d4/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/server/MiniYARNCluster.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/server/MiniYARNCluster.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/server/MiniYARNCluster.java
index 024adc6..74b7732 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/server/MiniYARNCluster.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/server/MiniYARNCluster.java
@@ -41,6 +41,7 @@ import org.apache.hadoop.util.Shell;
 import org.apache.hadoop.util.Shell.ShellCommandExecutor;
 import org.apache.hadoop.yarn.api.protocolrecords.GetClusterMetricsRequest;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
+import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.conf.HAUtil;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.event.Dispatcher;
@@ -67,6 +68,7 @@ import org.apache.hadoop.yarn.server.nodemanager.NodeStatusUpdater;
 import org.apache.hadoop.yarn.server.nodemanager.NodeStatusUpdaterImpl;
 import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager;
 import org.apache.hadoop.yarn.server.resourcemanager.ResourceTrackerService;
+import org.apache.hadoop.yarn.server.resourcemanager.reservation.ReservationSystemTestUtil;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptEventType;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event.RMAppAttemptRegistrationEvent;
@@ -76,6 +78,7 @@ import org.apache.hadoop.yarn.server.timeline.TimelineStore;
 import org.apache.hadoop.yarn.server.timeline.recovery.MemoryTimelineStateStore;
 import org.apache.hadoop.yarn.server.timeline.recovery.TimelineStateStore;
 import org.apache.hadoop.yarn.util.timeline.TimelineUtils;
+import org.apache.hadoop.yarn.util.resource.Resources;
 import org.apache.hadoop.yarn.webapp.util.WebAppUtils;
 
 import com.google.common.annotations.VisibleForTesting;
@@ -275,6 +278,12 @@ public class MiniYARNCluster extends CompositeService {
         conf instanceof YarnConfiguration ? conf : new YarnConfiguration(conf));
   }
 
+  @Override
+  protected synchronized void serviceStart() throws Exception {
+    super.serviceStart();
+    this.waitForNodeManagersToConnect(5000);
+  }
+
   private void setNonHARMConfigurationWithEphemeralPorts(Configuration conf) {
     String hostname = MiniYARNCluster.getHostname();
     conf.set(YarnConfiguration.RM_ADDRESS, hostname + ":0");
@@ -314,19 +323,7 @@ public class MiniYARNCluster extends CompositeService {
 
   private synchronized void startResourceManager(final int index) {
     try {
-      Thread rmThread = new Thread() {
-        public void run() {
-          resourceManagers[index].start();
-        }
-      };
-      rmThread.setName("RM-" + index);
-      rmThread.start();
-      int waitCount = 0;
-      while (resourceManagers[index].getServiceState() == STATE.INITED
-          && waitCount++ < 60) {
-        LOG.info("Waiting for RM to start...");
-        Thread.sleep(1500);
-      }
+      resourceManagers[index].start();
       if (resourceManagers[index].getServiceState() != STATE.STARTED) {
         // RM could have failed.
         throw new IOException(
@@ -456,6 +453,11 @@ public class MiniYARNCluster extends CompositeService {
     @Override
     protected synchronized void serviceStart() throws Exception {
       startResourceManager(index);
+      if(index == 0) {
+        resourceManagers[index].getRMContext().getRMAdminService()
+          .transitionToActive(new HAServiceProtocol.StateChangeRequestInfo(
+            HAServiceProtocol.RequestSource.REQUEST_BY_USER_FORCED));
+      }
       Configuration conf = resourceManagers[index].getConfig();
       LOG.info("Starting resourcemanager " + index);
       LOG.info("MiniYARN ResourceManager address: " +
@@ -565,26 +567,12 @@ public class MiniYARNCluster extends CompositeService {
     }
 
     protected synchronized void serviceStart() throws Exception {
-      try {
-        new Thread() {
-          public void run() {
-            nodeManagers[index].start();
-          }
-        }.start();
-        int waitCount = 0;
-        while (nodeManagers[index].getServiceState() == STATE.INITED
-            && waitCount++ < 60) {
-          LOG.info("Waiting for NM " + index + " to start...");
-          Thread.sleep(1000);
-        }
-        if (nodeManagers[index].getServiceState() != STATE.STARTED) {
-          // RM could have failed.
-          throw new IOException("NodeManager " + index + " failed to start");
-        }
-        super.serviceStart();
-      } catch (Throwable t) {
-        throw new YarnRuntimeException(t);
+      nodeManagers[index].start();
+      if (nodeManagers[index].getServiceState() != STATE.STARTED) {
+        // NM could have failed.
+        throw new IOException("NodeManager " + index + " failed to start");
       }
+      super.serviceStart();
     }
 
     @Override
@@ -715,7 +703,7 @@ public class MiniYARNCluster extends CompositeService {
   /**
    * Wait for all the NodeManagers to connect to the ResourceManager.
    *
-   * @param timeout Time to wait (sleeps in 100 ms intervals) in milliseconds.
+   * @param timeout Time to wait (sleeps in 10 ms intervals) in milliseconds.
    * @return true if all NodeManagers connect to the (Active)
    * ResourceManager, false otherwise.
    * @throws YarnException
@@ -724,17 +712,19 @@ public class MiniYARNCluster extends CompositeService {
   public boolean waitForNodeManagersToConnect(long timeout)
       throws YarnException, InterruptedException {
     GetClusterMetricsRequest req = GetClusterMetricsRequest.newInstance();
-    for (int i = 0; i < timeout / 100; i++) {
+    for (int i = 0; i < timeout / 10; i++) {
       ResourceManager rm = getResourceManager();
       if (rm == null) {
         throw new YarnException("Can not find the active RM.");
       }
       else if (nodeManagers.length == rm.getClientRMService()
-            .getClusterMetrics(req).getClusterMetrics().getNumNodeManagers()) {
+          .getClusterMetrics(req).getClusterMetrics().getNumNodeManagers()) {
+        LOG.info("All Node Managers connected in MiniYARNCluster");
         return true;
       }
-      Thread.sleep(100);
+      Thread.sleep(10);
     }
+    LOG.info("Node Managers did not connect within 5000ms");
     return false;
   }
 
@@ -769,18 +759,7 @@ public class MiniYARNCluster extends CompositeService {
 
     @Override
     protected synchronized void serviceStart() throws Exception {
-
-      new Thread() {
-        public void run() {
-          appHistoryServer.start();
-        };
-      }.start();
-      int waitCount = 0;
-      while (appHistoryServer.getServiceState() == STATE.INITED
-          && waitCount++ < 60) {
-        LOG.info("Waiting for Timeline Server to start...");
-        Thread.sleep(1500);
-      }
+      appHistoryServer.start();
       if (appHistoryServer.getServiceState() != STATE.STARTED) {
         // AHS could have failed.
         IOException ioe = new IOException(

http://git-wip-us.apache.org/repos/asf/hadoop/blob/92b7e0d4/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/server/TestMiniYARNClusterForHA.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/server/TestMiniYARNClusterForHA.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/server/TestMiniYARNClusterForHA.java
index e84d62e..384d1cd 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/server/TestMiniYARNClusterForHA.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/server/TestMiniYARNClusterForHA.java
@@ -44,10 +44,6 @@ public class TestMiniYARNClusterForHA {
     cluster.init(conf);
     cluster.start();
 
-    cluster.getResourceManager(0).getRMContext().getRMAdminService()
-        .transitionToActive(new HAServiceProtocol.StateChangeRequestInfo(
-            HAServiceProtocol.RequestSource.REQUEST_BY_USER));
-
     assertFalse("RM never turned active", -1 == cluster.getActiveRMIndex());
   }
 


[43/46] hadoop git commit: HDFS-9874. Long living DataXceiver threads cause volume shutdown to block. Contributed by Rushabh Shah.

Posted by ae...@apache.org.
HDFS-9874. Long living DataXceiver threads cause volume shutdown to block. Contributed by Rushabh Shah.


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

Branch: refs/heads/HDFS-7240
Commit: 63c966a3fbeb675959fc4101e65de9f57aecd17d
Parents: dc951e6
Author: Kihwal Lee <ki...@apache.org>
Authored: Fri Mar 18 10:24:59 2016 -0500
Committer: Kihwal Lee <ki...@apache.org>
Committed: Fri Mar 18 10:24:59 2016 -0500

----------------------------------------------------------------------
 .../hdfs/server/datanode/ReplicaInPipeline.java |  7 +++
 .../datanode/fsdataset/impl/FsDatasetImpl.java  | 13 ++++
 .../datanode/fsdataset/impl/FsVolumeImpl.java   |  6 ++
 .../fsdataset/impl/TestFsDatasetImpl.java       | 66 ++++++++++++++++++++
 4 files changed, 92 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/63c966a3/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ReplicaInPipeline.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ReplicaInPipeline.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ReplicaInPipeline.java
index d9406f0..5caca15 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ReplicaInPipeline.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ReplicaInPipeline.java
@@ -183,6 +183,13 @@ public class ReplicaInPipeline extends ReplicaInfo
     this.writer = writer;
   }
   
+  public void interruptThread() {
+    if (writer != null && writer != Thread.currentThread() 
+        && writer.isAlive()) {
+      this.writer.interrupt();
+    }
+  }
+
   @Override  // Object
   public boolean equals(Object o) {
     return super.equals(o);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/63c966a3/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 2e8226a..d6a0df6 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
@@ -3112,5 +3112,18 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
   public void setTimer(Timer newTimer) {
     this.timer = newTimer;
   }
+
+  synchronized void stopAllDataxceiverThreads(FsVolumeImpl volume) {
+    for (String blockPoolId : volumeMap.getBlockPoolList()) {
+      Collection<ReplicaInfo> replicas = volumeMap.replicas(blockPoolId);
+      for (ReplicaInfo replicaInfo : replicas) {
+        if (replicaInfo instanceof ReplicaInPipeline
+            && replicaInfo.getVolume().equals(volume)) {
+          ReplicaInPipeline replicaInPipeline = (ReplicaInPipeline) replicaInfo;
+          replicaInPipeline.interruptThread();
+        }
+      }
+    }
+  }
 }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/63c966a3/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsVolumeImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsVolumeImpl.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsVolumeImpl.java
index 857d0ad..0d060f9 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsVolumeImpl.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsVolumeImpl.java
@@ -240,6 +240,11 @@ public class FsVolumeImpl implements FsVolumeSpi {
     Preconditions.checkState(reference.getReferenceCount() > 0);
   }
 
+  @VisibleForTesting
+  int getReferenceCount() {
+    return this.reference.getReferenceCount();
+  }
+
   /**
    * Close this volume.
    * @throws IOException if the volume is closed.
@@ -247,6 +252,7 @@ public class FsVolumeImpl implements FsVolumeSpi {
   void setClosed() throws IOException {
     try {
       this.reference.setClosed();
+      dataset.stopAllDataxceiverThreads(this);
     } catch (ClosedChannelException e) {
       throw new IOException("The volume has already closed.", e);
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/63c966a3/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestFsDatasetImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestFsDatasetImpl.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestFsDatasetImpl.java
index b2cfe89..70e9332 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestFsDatasetImpl.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestFsDatasetImpl.java
@@ -21,14 +21,19 @@ import com.google.common.collect.Lists;
 
 import org.apache.commons.io.FileUtils;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileSystemTestHelper;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.StorageType;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
+import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.common.Storage;
 import org.apache.hadoop.hdfs.server.common.StorageInfo;
@@ -51,6 +56,7 @@ import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.util.DiskChecker;
 import org.apache.hadoop.util.FakeTimer;
 import org.apache.hadoop.util.StringUtils;
+import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
 import org.mockito.Matchers;
@@ -622,4 +628,64 @@ public class TestFsDatasetImpl {
     LOG.info("Volumes removed");
     brReceivedLatch.await();
   }
+
+  /**
+   * Tests stopping all the active DataXceiver thread on volume failure event.
+   * @throws Exception
+   */
+  @Test
+  public void testCleanShutdownOfVolume() throws Exception {
+    MiniDFSCluster cluster = null;
+    try {
+      Configuration config = new HdfsConfiguration();
+      config.setLong(
+          DFSConfigKeys.DFS_DATANODE_XCEIVER_STOP_TIMEOUT_MILLIS_KEY, 1000);
+      config.setInt(DFSConfigKeys.DFS_DATANODE_FAILED_VOLUMES_TOLERATED_KEY, 1);
+
+      cluster = new MiniDFSCluster.Builder(config).numDataNodes(1).build();
+      cluster.waitActive();
+      FileSystem fs = cluster.getFileSystem();
+      DataNode dataNode = cluster.getDataNodes().get(0);
+      Path filePath = new Path("test.dat");
+      // Create a file and keep the output stream unclosed.
+      FSDataOutputStream out = fs.create(filePath, (short) 1);
+      out.write(1);
+      out.hflush();
+
+      ExtendedBlock block = DFSTestUtil.getFirstBlock(fs, filePath);
+      FsVolumeImpl volume = (FsVolumeImpl) dataNode.getFSDataset().getVolume(
+          block);
+      File finalizedDir = volume.getFinalizedDir(cluster.getNamesystem()
+          .getBlockPoolId());
+
+      if (finalizedDir.exists()) {
+        // Remove write and execute access so that checkDiskErrorThread detects
+        // this volume is bad.
+        finalizedDir.setExecutable(false);
+        finalizedDir.setWritable(false);
+      }
+      Assert.assertTrue("Reference count for the volume should be greater "
+          + "than 0", volume.getReferenceCount() > 0);
+      // Invoke the synchronous checkDiskError method
+      dataNode.getFSDataset().checkDataDir();
+      // Sleep for 1 second so that datanode can interrupt and cluster clean up
+      Thread.sleep(1000);
+      assertEquals("There are active threads still referencing volume: "
+          + volume.getBasePath(), 0, volume.getReferenceCount());
+      LocatedBlock lb = DFSTestUtil.getAllBlocks(fs, filePath).get(0);
+      DatanodeInfo info = lb.getLocations()[0];
+
+      try {
+        out.close();
+        Assert.fail("This is not a valid code path. "
+            + "out.close should have thrown an exception.");
+      } catch (IOException ioe) {
+        Assert.assertTrue(ioe.getMessage().contains(info.toString()));
+      }
+      finalizedDir.setWritable(true);
+      finalizedDir.setExecutable(true);
+    } finally {
+    cluster.shutdown();
+    }
+  }
 }


[06/46] hadoop git commit: HDFS-9942. Add an HTrace span when refreshing the groups for a username (cmccabe)

Posted by ae...@apache.org.
HDFS-9942. Add an HTrace span when refreshing the groups for a username (cmccabe)


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

Branch: refs/heads/HDFS-7240
Commit: 6e9a582eb1ab040f532fbe95694f5272cda34188
Parents: 7542996
Author: Colin Patrick Mccabe <cm...@cloudera.com>
Authored: Fri Mar 11 12:26:40 2016 -0800
Committer: Colin Patrick Mccabe <cm...@cloudera.com>
Committed: Fri Mar 11 12:26:40 2016 -0800

----------------------------------------------------------------------
 .../java/org/apache/hadoop/security/Groups.java    | 17 ++++++++++++++++-
 1 file changed, 16 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/6e9a582e/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/Groups.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/Groups.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/Groups.java
index 9fd39b0..11d8f2a 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/Groups.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/Groups.java
@@ -27,6 +27,8 @@ import java.util.Set;
 import java.util.concurrent.ExecutionException;
 import java.util.concurrent.TimeUnit;
 
+import org.apache.htrace.core.TraceScope;
+import org.apache.htrace.core.Tracer;
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Ticker;
 import com.google.common.cache.CacheBuilder;
@@ -217,7 +219,20 @@ public class Groups {
      */
     @Override
     public List<String> load(String user) throws Exception {
-      List<String> groups = fetchGroupList(user);
+      TraceScope scope = null;
+      Tracer tracer = Tracer.curThreadTracer();
+      if (tracer != null) {
+        scope = tracer.newScope("Groups#fetchGroupList");
+        scope.addKVAnnotation("user", user);
+      }
+      List<String> groups = null;
+      try {
+        groups = fetchGroupList(user);
+      } finally {
+        if (scope != null) {
+          scope.close();
+        }
+      }
 
       if (groups.isEmpty()) {
         if (isNegativeCacheEnabled()) {


[40/46] hadoop git commit: YARN-4812. TestFairScheduler#testContinuousScheduling fails intermittently. (kasha)

Posted by ae...@apache.org.
YARN-4812. TestFairScheduler#testContinuousScheduling fails intermittently. (kasha)


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

Branch: refs/heads/HDFS-7240
Commit: f84af8bd588763c4e99305742d8c86ed596e8359
Parents: 80fa70c
Author: Karthik Kambatla <ka...@apache.org>
Authored: Thu Mar 17 05:54:06 2016 -0700
Committer: Karthik Kambatla <ka...@apache.org>
Committed: Thu Mar 17 05:54:06 2016 -0700

----------------------------------------------------------------------
 .../scheduler/fair/FairSchedulerTestBase.java   | 22 +++++-
 .../fair/TestContinuousScheduling.java          | 78 ++++++++++++++++++--
 .../scheduler/fair/TestFairScheduler.java       | 75 -------------------
 3 files changed, 91 insertions(+), 84 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/f84af8bd/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FairSchedulerTestBase.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/fair/FairSchedulerTestBase.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FairSchedulerTestBase.java
index 3caeb3c..0e1d904 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FairSchedulerTestBase.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FairSchedulerTestBase.java
@@ -17,6 +17,7 @@
  */
 package org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair;
 
+import org.junit.Assert;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.when;
 
@@ -50,7 +51,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAddedSchedulerEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAttemptAddedSchedulerEvent;
 import org.apache.hadoop.yarn.server.utils.BuilderUtils;
-import org.apache.hadoop.yarn.util.Clock;
+import org.apache.hadoop.yarn.util.resource.Resources;
 
 public class FairSchedulerTestBase {
   public final static String TEST_DIR =
@@ -66,6 +67,8 @@ public class FairSchedulerTestBase {
   protected FairScheduler scheduler;
   protected ResourceManager resourceManager;
   public static final float TEST_RESERVATION_THRESHOLD = 0.09f;
+  private static final int SLEEP_DURATION = 10;
+  private static final int SLEEP_RETRIES = 1000;
 
   // Helper methods
   public Configuration createConfiguration() {
@@ -260,4 +263,21 @@ public class FairSchedulerTestBase {
         .put(attemptId.getApplicationId(), app);
     return app;
   }
+
+  protected void checkAppConsumption(FSAppAttempt app, Resource resource)
+      throws InterruptedException {
+    for (int i = 0; i < SLEEP_RETRIES; i++) {
+      if (Resources.equals(resource, app.getCurrentConsumption())) {
+        break;
+      } else {
+        Thread.sleep(SLEEP_DURATION);
+      }
+    }
+
+    // available resource
+    Assert.assertEquals(resource.getMemory(),
+        app.getCurrentConsumption().getMemory());
+    Assert.assertEquals(resource.getVirtualCores(),
+        app.getCurrentConsumption().getVirtualCores());
+  }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f84af8bd/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestContinuousScheduling.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/fair/TestContinuousScheduling.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestContinuousScheduling.java
index 65c80a6..2e7b3f8 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestContinuousScheduling.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestContinuousScheduling.java
@@ -21,9 +21,11 @@ package org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair;
 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.ResourceRequest;
 import org.apache.hadoop.yarn.server.resourcemanager.MockNodes;
 import org.apache.hadoop.yarn.server.resourcemanager.MockRM;
+import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeAddedSchedulerEvent;
 
@@ -31,13 +33,17 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeUpdateS
 import org.apache.hadoop.yarn.util.ControlledClock;
 import org.apache.hadoop.yarn.util.resource.Resources;
 import org.junit.After;
+import org.junit.Assert;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 import org.junit.Before;
 import org.junit.Test;
 
 import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.Iterator;
 import java.util.List;
+import java.util.Set;
 
 public class TestContinuousScheduling extends FairSchedulerTestBase {
   private ControlledClock mockClock;
@@ -78,7 +84,7 @@ public class TestContinuousScheduling extends FairSchedulerTestBase {
   }
 
   @Test (timeout = 60000)
-  public void testSchedulingDelay() throws InterruptedException {
+  public void testBasic() throws InterruptedException {
     // Add one node
     String host = "127.0.0.1";
     RMNode node1 = MockNodes.newNodeInfo(
@@ -88,8 +94,6 @@ public class TestContinuousScheduling extends FairSchedulerTestBase {
     NodeUpdateSchedulerEvent nodeUpdateEvent = new NodeUpdateSchedulerEvent(node1);
     scheduler.handle(nodeUpdateEvent);
 
-    // Create one application and submit one each of node-local, rack-local
-    // and ANY requests
     ApplicationAttemptId appAttemptId =
         createAppAttemptId(this.APP_ID++, this.ATTEMPT_ID++);
     createMockRMApp(appAttemptId);
@@ -102,11 +106,69 @@ public class TestContinuousScheduling extends FairSchedulerTestBase {
         appAttemptId, ask, new ArrayList<ContainerId>(), null, null, null, null);
     FSAppAttempt app = scheduler.getSchedulerApp(appAttemptId);
 
-    // Advance time and let continuous scheduling kick in
-    mockClock.tickSec(1);
-    while (1024 != app.getCurrentConsumption().getMemory()) {
-      Thread.sleep(100);
+    triggerSchedulingAttempt();
+    checkAppConsumption(app, Resources.createResource(1024, 1));
+  }
+
+  @Test (timeout = 10000)
+  public void testSortedNodes() throws Exception {
+    // Add two nodes
+    RMNode node1 =
+        MockNodes.newNodeInfo(1, Resources.createResource(8 * 1024, 8), 1,
+            "127.0.0.1");
+    NodeAddedSchedulerEvent nodeEvent1 = new NodeAddedSchedulerEvent(node1);
+    scheduler.handle(nodeEvent1);
+    RMNode node2 =
+        MockNodes.newNodeInfo(1, Resources.createResource(8 * 1024, 8), 2,
+            "127.0.0.2");
+    NodeAddedSchedulerEvent nodeEvent2 = new NodeAddedSchedulerEvent(node2);
+    scheduler.handle(nodeEvent2);
+
+    // available resource
+    Assert.assertEquals(scheduler.getClusterResource().getMemory(), 16 * 1024);
+    Assert.assertEquals(scheduler.getClusterResource().getVirtualCores(), 16);
+
+    // send application request
+    ApplicationAttemptId appAttemptId =
+        createAppAttemptId(this.APP_ID++, this.ATTEMPT_ID++);
+    createMockRMApp(appAttemptId);
+
+    scheduler.addApplication(appAttemptId.getApplicationId(),
+        "queue11", "user11", false);
+    scheduler.addApplicationAttempt(appAttemptId, false, false);
+    List<ResourceRequest> ask = new ArrayList<>();
+    ResourceRequest request =
+        createResourceRequest(1024, 1, ResourceRequest.ANY, 1, 1, true);
+    ask.add(request);
+    scheduler.allocate(appAttemptId, ask,
+        new ArrayList<ContainerId>(), null, null, null, null);
+    triggerSchedulingAttempt();
+
+    FSAppAttempt app = scheduler.getSchedulerApp(appAttemptId);
+    checkAppConsumption(app, Resources.createResource(1024, 1));
+
+    // another request
+    request =
+        createResourceRequest(1024, 1, ResourceRequest.ANY, 2, 1, true);
+    ask.clear();
+    ask.add(request);
+    scheduler.allocate(appAttemptId, ask,
+        new ArrayList<ContainerId>(), null, null, null, null);
+    triggerSchedulingAttempt();
+
+    checkAppConsumption(app, Resources.createResource(2048,2));
+
+    // 2 containers should be assigned to 2 nodes
+    Set<NodeId> nodes = new HashSet<NodeId>();
+    Iterator<RMContainer> it = app.getLiveContainers().iterator();
+    while (it.hasNext()) {
+      nodes.add(it.next().getContainer().getNodeId());
     }
-    assertEquals(1024, app.getCurrentConsumption().getMemory());
+    Assert.assertEquals(2, nodes.size());
+  }
+
+  private void triggerSchedulingAttempt() {
+    mockClock.tickMsec(
+        2 * scheduler.getConf().getContinuousSchedulingSleepMs());
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f84af8bd/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFairScheduler.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/fair/TestFairScheduler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFairScheduler.java
index 1add193..a75b5ce 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFairScheduler.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFairScheduler.java
@@ -3738,81 +3738,6 @@ public class TestFairScheduler extends FairSchedulerTestBase {
     verifyQueueNumRunnable("queue1.sub3", 0, 0);
   }
 
-  @Test (timeout = 10000)
-  public void testContinuousScheduling() throws Exception {
-    // set continuous scheduling enabled
-    scheduler = new FairScheduler();
-    Configuration conf = createConfiguration();
-    conf.setBoolean(FairSchedulerConfiguration.CONTINUOUS_SCHEDULING_ENABLED,
-            true);
-    scheduler.setRMContext(resourceManager.getRMContext());
-    scheduler.init(conf);
-    scheduler.start();
-    scheduler.reinitialize(conf, resourceManager.getRMContext());
-    Assert.assertTrue("Continuous scheduling should be enabled.",
-        scheduler.isContinuousSchedulingEnabled());
-
-    // Add two nodes
-    RMNode node1 =
-            MockNodes.newNodeInfo(1, Resources.createResource(8 * 1024, 8), 1,
-                    "127.0.0.1");
-    NodeAddedSchedulerEvent nodeEvent1 = new NodeAddedSchedulerEvent(node1);
-    scheduler.handle(nodeEvent1);
-    RMNode node2 =
-            MockNodes.newNodeInfo(1, Resources.createResource(8 * 1024, 8), 2,
-                    "127.0.0.2");
-    NodeAddedSchedulerEvent nodeEvent2 = new NodeAddedSchedulerEvent(node2);
-    scheduler.handle(nodeEvent2);
-
-    // available resource
-    Assert.assertEquals(scheduler.getClusterResource().getMemory(), 16 * 1024);
-    Assert.assertEquals(scheduler.getClusterResource().getVirtualCores(), 16);
-
-    // send application request
-    ApplicationAttemptId appAttemptId =
-            createAppAttemptId(this.APP_ID++, this.ATTEMPT_ID++);
-    createMockRMApp(appAttemptId);
-
-    scheduler.addApplication(appAttemptId.getApplicationId(), "queue11", "user11", false);
-    scheduler.addApplicationAttempt(appAttemptId, false, false);
-    List<ResourceRequest> ask = new ArrayList<ResourceRequest>();
-    ResourceRequest request =
-            createResourceRequest(1024, 1, ResourceRequest.ANY, 1, 1, true);
-    ask.add(request);
-    scheduler.allocate(appAttemptId, ask, new ArrayList<ContainerId>(), null, null, null, null);
-
-    // waiting for continuous_scheduler_sleep_time
-    // at least one pass
-    Thread.sleep(scheduler.getConf().getContinuousSchedulingSleepMs() + 500);
-
-    FSAppAttempt app = scheduler.getSchedulerApp(appAttemptId);
-    // Wait until app gets resources.
-    while (app.getCurrentConsumption().equals(Resources.none())) { }
-
-    // check consumption
-    Assert.assertEquals(1024, app.getCurrentConsumption().getMemory());
-    Assert.assertEquals(1, app.getCurrentConsumption().getVirtualCores());
-
-    // another request
-    request =
-            createResourceRequest(1024, 1, ResourceRequest.ANY, 2, 1, true);
-    ask.clear();
-    ask.add(request);
-    scheduler.stop();
-    scheduler.allocate(appAttemptId, ask, new ArrayList<ContainerId>(), null, null, null, null);
-    scheduler.continuousSchedulingAttempt();
-    Assert.assertEquals(2048, app.getCurrentConsumption().getMemory());
-    Assert.assertEquals(2, app.getCurrentConsumption().getVirtualCores());
-
-    // 2 containers should be assigned to 2 nodes
-    Set<NodeId> nodes = new HashSet<NodeId>();
-    Iterator<RMContainer> it = app.getLiveContainers().iterator();
-    while (it.hasNext()) {
-      nodes.add(it.next().getContainer().getNodeId());
-    }
-    Assert.assertEquals(2, nodes.size());
-  }
-
   @Test
   public void testContinuousSchedulingWithNodeRemoved() throws Exception {
     // Disable continuous scheduling, will invoke continuous scheduling once manually


[31/46] hadoop git commit: CapacityScheduler: Improve preemption to only kill containers that would satisfy the incoming request. (Wangda Tan)

Posted by ae...@apache.org.
CapacityScheduler: Improve preemption to only kill containers that would satisfy the incoming request. (Wangda Tan)


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

Branch: refs/heads/HDFS-7240
Commit: 7e8c9beb4156dcaeb3a11e60aaa06d2370626913
Parents: 32d043d
Author: Wangda Tan <wa...@apache.org>
Authored: Wed Mar 16 16:59:59 2016 -0700
Committer: Wangda Tan <wa...@apache.org>
Committed: Wed Mar 16 16:59:59 2016 -0700

----------------------------------------------------------------------
 .../ProportionalCapacityPreemptionPolicy.java   | 166 +++--
 .../rmcontainer/RMContainer.java                |   1 +
 .../scheduler/PreemptableResourceScheduler.java |   2 +-
 .../scheduler/ResourceLimits.java               |   9 +
 .../scheduler/SchedulerNode.java                |   9 +-
 .../scheduler/capacity/AbstractCSQueue.java     |  45 +-
 .../scheduler/capacity/CSAssignment.java        |  11 +
 .../scheduler/capacity/CapacityScheduler.java   | 132 +++-
 .../CapacitySchedulerConfiguration.java         |  14 +-
 .../capacity/CapacitySchedulerContext.java      |  15 +-
 .../scheduler/capacity/LeafQueue.java           |  69 ++
 .../scheduler/capacity/ParentQueue.java         | 157 ++++-
 .../allocator/AbstractContainerAllocator.java   |   2 +
 .../capacity/allocator/ContainerAllocation.java |  12 +
 .../allocator/RegularContainerAllocator.java    |  39 +-
 .../capacity/preemption/KillableContainer.java  |  45 ++
 .../capacity/preemption/PreemptableQueue.java   | 102 +++
 .../capacity/preemption/PreemptionManager.java  | 165 +++++
 .../scheduler/common/AssignmentInformation.java |   6 +-
 .../scheduler/common/fica/FiCaSchedulerApp.java |  32 +-
 .../common/fica/FiCaSchedulerNode.java          |  65 +-
 .../scheduler/event/SchedulerEventType.java     |  15 +-
 .../resourcemanager/TestRMDispatcher.java       |   4 +-
 .../server/resourcemanager/TestRMRestart.java   |   2 +-
 .../applicationsmanager/TestAMRestart.java      |   7 +-
 ...estProportionalCapacityPreemptionPolicy.java |   6 +-
 ...pacityPreemptionPolicyForNodePartitions.java |   2 +
 .../capacity/TestApplicationLimits.java         |   2 +
 .../capacity/TestApplicationPriority.java       |   6 +-
 .../capacity/TestCapacityScheduler.java         |   8 +-
 .../TestCapacitySchedulerPreemption.java        | 677 +++++++++++++++++++
 .../scheduler/capacity/TestChildQueueOrder.java |   2 +
 .../scheduler/capacity/TestLeafQueue.java       |   3 +
 .../TestNodeLabelContainerAllocation.java       |  97 +++
 .../scheduler/capacity/TestParentQueue.java     |   2 +
 .../scheduler/capacity/TestReservations.java    |   2 +
 .../scheduler/capacity/TestUtils.java           |  36 +
 .../fair/TestFairSchedulerPreemption.java       |   2 +-
 38 files changed, 1785 insertions(+), 186 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/7e8c9beb/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/ProportionalCapacityPreemptionPolicy.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/monitor/capacity/ProportionalCapacityPreemptionPolicy.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/ProportionalCapacityPreemptionPolicy.java
index 3a87edb..9b499c8 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/ProportionalCapacityPreemptionPolicy.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/ProportionalCapacityPreemptionPolicy.java
@@ -35,7 +35,7 @@ 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.ApplicationAttemptId;
-import org.apache.hadoop.yarn.api.records.NodeId;
+import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.Priority;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
@@ -49,6 +49,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.Capacity
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.LeafQueue;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueueCapacities;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.preemption.PreemptableQueue;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerApp;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.ContainerPreemptEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEventType;
@@ -125,8 +126,8 @@ public class ProportionalCapacityPreemptionPolicy implements SchedulingEditPolic
   private long maxWaitTime;
   private CapacityScheduler scheduler;
   private long monitoringInterval;
-  private final Map<RMContainer,Long> preempted =
-    new HashMap<RMContainer,Long>();
+  private final Map<RMContainer, Long> preempted = new HashMap<>();
+
   private ResourceCalculator rc;
   private float percentageClusterPreemptionAllowed;
   private double naturalTerminationFactor;
@@ -135,6 +136,10 @@ public class ProportionalCapacityPreemptionPolicy implements SchedulingEditPolic
       new HashMap<>();
   private RMNodeLabelsManager nlm;
 
+  // Preemptable Entities, synced from scheduler at every run
+  private Map<String, PreemptableQueue> preemptableEntities = null;
+  private Set<ContainerId> killableContainers;
+
   public ProportionalCapacityPreemptionPolicy() {
     clock = SystemClock.getInstance();
   }
@@ -184,6 +189,64 @@ public class ProportionalCapacityPreemptionPolicy implements SchedulingEditPolic
     Resource clusterResources = Resources.clone(scheduler.getClusterResource());
     containerBasedPreemptOrKill(root, clusterResources);
   }
+
+  @SuppressWarnings("unchecked")
+  private void cleanupStaledKillableContainers(Resource cluster,
+      Set<String> leafQueueNames) {
+    for (String q : leafQueueNames) {
+      for (TempQueuePerPartition tq : getQueuePartitions(q)) {
+        // When queue's used - killable <= guaranteed and, killable > 0, we need
+        // to check if any of killable containers needs to be reverted
+        if (Resources.lessThanOrEqual(rc, cluster,
+            Resources.subtract(tq.current, tq.killable), tq.idealAssigned)
+            && Resources.greaterThan(rc, cluster, tq.killable, Resources.none())) {
+          // How many killable resources need to be reverted
+          // need-to-revert = already-marked-killable - (current - ideal)
+          Resource toBeRevertedFromKillable = Resources.subtract(tq.killable,
+              Resources.subtract(tq.current, tq.idealAssigned));
+
+          Resource alreadyReverted = Resources.createResource(0);
+
+          for (RMContainer c : preemptableEntities.get(q).getKillableContainers(
+              tq.partition).values()) {
+            if (Resources.greaterThanOrEqual(rc, cluster, alreadyReverted,
+                toBeRevertedFromKillable)) {
+              break;
+            }
+
+            if (Resources.greaterThan(rc, cluster,
+                Resources.add(alreadyReverted, c.getAllocatedResource()),
+                toBeRevertedFromKillable)) {
+              continue;
+            } else {
+              // This container need to be marked to unkillable
+              Resources.addTo(alreadyReverted, c.getAllocatedResource());
+              rmContext.getDispatcher().getEventHandler().handle(
+                  new ContainerPreemptEvent(c.getApplicationAttemptId(), c,
+                      SchedulerEventType.MARK_CONTAINER_FOR_NONKILLABLE));
+            }
+          }
+
+        }
+      }
+    }
+  }
+
+  private void syncKillableContainersFromScheduler() {
+    // sync preemptable entities from scheduler
+    preemptableEntities =
+        scheduler.getPreemptionManager().getShallowCopyOfPreemptableEntities();
+
+    killableContainers = new HashSet<>();
+    for (Map.Entry<String, PreemptableQueue> entry : preemptableEntities
+        .entrySet()) {
+      PreemptableQueue entity = entry.getValue();
+      for (Map<ContainerId, RMContainer> map : entity.getKillableContainers()
+          .values()) {
+        killableContainers.addAll(map.keySet());
+      }
+    }
+  }
   
   /**
    * This method selects and tracks containers to be preempted. If a container
@@ -201,6 +264,8 @@ public class ProportionalCapacityPreemptionPolicy implements SchedulingEditPolic
         .getNodeLabelManager().getClusterNodeLabelNames());
     allPartitions.add(RMNodeLabelsManager.NO_LABEL);
 
+    syncKillableContainersFromScheduler();
+
     // extract a summary of the queues from scheduler
     synchronized (scheduler) {
       queueToPartitions.clear();
@@ -228,13 +293,17 @@ public class ProportionalCapacityPreemptionPolicy implements SchedulingEditPolic
           recursivelyComputeIdealAssignment(tRoot, totalPreemptionAllowed);
     }
 
+    // remove containers from killable list when we want to preempt less resources
+    // from queue.
+    cleanupStaledKillableContainers(clusterResources, leafQueueNames);
+
     // based on ideal allocation select containers to be preempted from each
     // queue and each application
     Map<ApplicationAttemptId,Set<RMContainer>> toPreempt =
         getContainersToPreempt(leafQueueNames, clusterResources);
 
     if (LOG.isDebugEnabled()) {
-      logToCSV(new ArrayList<String>(leafQueueNames));
+      logToCSV(new ArrayList<>(leafQueueNames));
     }
 
     // if we are in observeOnly mode return before any action is taken
@@ -254,10 +323,10 @@ public class ProportionalCapacityPreemptionPolicy implements SchedulingEditPolic
         // if we tried to preempt this for more than maxWaitTime
         if (preempted.get(container) != null &&
             preempted.get(container) + maxWaitTime < clock.getTime()) {
-          // kill it
+          // mark container killable
           rmContext.getDispatcher().getEventHandler().handle(
               new ContainerPreemptEvent(appAttemptId, container,
-                  SchedulerEventType.KILL_PREEMPTED_CONTAINER));
+                  SchedulerEventType.MARK_CONTAINER_FOR_KILLABLE));
           preempted.remove(container);
         } else {
           if (preempted.get(container) != null) {
@@ -333,14 +402,14 @@ public class ProportionalCapacityPreemptionPolicy implements SchedulingEditPolic
 
     // qAlloc tracks currently active queues (will decrease progressively as
     // demand is met)
-    List<TempQueuePerPartition> qAlloc = new ArrayList<TempQueuePerPartition>(queues);
+    List<TempQueuePerPartition> qAlloc = new ArrayList<>(queues);
     // unassigned tracks how much resources are still to assign, initialized
     // with the total capacity for this set of queues
     Resource unassigned = Resources.clone(tot_guarant);
 
     // group queues based on whether they have non-zero guaranteed capacity
-    Set<TempQueuePerPartition> nonZeroGuarQueues = new HashSet<TempQueuePerPartition>();
-    Set<TempQueuePerPartition> zeroGuarQueues = new HashSet<TempQueuePerPartition>();
+    Set<TempQueuePerPartition> nonZeroGuarQueues = new HashSet<>();
+    Set<TempQueuePerPartition> zeroGuarQueues = new HashSet<>();
 
     for (TempQueuePerPartition q : qAlloc) {
       if (Resources
@@ -415,8 +484,8 @@ public class ProportionalCapacityPreemptionPolicy implements SchedulingEditPolic
     // idealAssigned >= current + pending), remove it from consideration.
     // Sort queues from most under-guaranteed to most over-guaranteed.
     TQComparator tqComparator = new TQComparator(rc, tot_guarant);
-    PriorityQueue<TempQueuePerPartition> orderedByNeed =
-        new PriorityQueue<TempQueuePerPartition>(10, tqComparator);
+    PriorityQueue<TempQueuePerPartition> orderedByNeed = new PriorityQueue<>(10,
+        tqComparator);
     for (Iterator<TempQueuePerPartition> i = qAlloc.iterator(); i.hasNext();) {
       TempQueuePerPartition q = i.next();
       if (Resources.greaterThan(rc, tot_guarant, q.current, q.guaranteed)) {
@@ -474,7 +543,7 @@ public class ProportionalCapacityPreemptionPolicy implements SchedulingEditPolic
   // percentage of guaranteed.
   protected Collection<TempQueuePerPartition> getMostUnderservedQueues(
       PriorityQueue<TempQueuePerPartition> orderedByNeed, TQComparator tqComparator) {
-    ArrayList<TempQueuePerPartition> underserved = new ArrayList<TempQueuePerPartition>();
+    ArrayList<TempQueuePerPartition> underserved = new ArrayList<>();
     while (!orderedByNeed.isEmpty()) {
       TempQueuePerPartition q1 = orderedByNeed.remove();
       underserved.add(q1);
@@ -502,7 +571,7 @@ public class ProportionalCapacityPreemptionPolicy implements SchedulingEditPolic
     
     if (ignoreGuar) {
       for (TempQueuePerPartition q : queues) {
-        q.normalizedGuarantee = (float)  1.0f / ((float) queues.size());
+        q.normalizedGuarantee = 1.0f / queues.size();
       }
     } else {
       for (TempQueuePerPartition q : queues) {
@@ -515,8 +584,9 @@ public class ProportionalCapacityPreemptionPolicy implements SchedulingEditPolic
     }
   }
 
-  private String getPartitionByNodeId(NodeId nodeId) {
-    return scheduler.getSchedulerNode(nodeId).getPartition();
+  private String getPartitionByRMContainer(RMContainer rmContainer) {
+    return scheduler.getSchedulerNode(rmContainer.getAllocatedNode())
+        .getPartition();
   }
 
   /**
@@ -534,7 +604,7 @@ public class ProportionalCapacityPreemptionPolicy implements SchedulingEditPolic
       return false;
     }
 
-    String nodePartition = getPartitionByNodeId(rmContainer.getAllocatedNode());
+    String nodePartition = getPartitionByRMContainer(rmContainer);
     Resource toObtainByPartition =
         resourceToObtainByPartitions.get(nodePartition);
 
@@ -575,7 +645,7 @@ public class ProportionalCapacityPreemptionPolicy implements SchedulingEditPolic
       ApplicationAttemptId appAttemptId, RMContainer containerToPreempt) {
     Set<RMContainer> set;
     if (null == (set = preemptMap.get(appAttemptId))) {
-      set = new HashSet<RMContainer>();
+      set = new HashSet<>();
       preemptMap.put(appAttemptId, set);
     }
     set.add(containerToPreempt);
@@ -587,7 +657,7 @@ public class ProportionalCapacityPreemptionPolicy implements SchedulingEditPolic
    * over-capacity queue. It uses {@link #NATURAL_TERMINATION_FACTOR} to
    * account for containers that will naturally complete.
    *
-   * @param queues set of leaf queues to preempt from
+   * @param leafQueueNames set of leaf queues to preempt from
    * @param clusterResource total amount of cluster resources
    * @return a map of applciationID to set of containers to preempt
    */
@@ -595,8 +665,8 @@ public class ProportionalCapacityPreemptionPolicy implements SchedulingEditPolic
       Set<String> leafQueueNames, Resource clusterResource) {
 
     Map<ApplicationAttemptId, Set<RMContainer>> preemptMap =
-        new HashMap<ApplicationAttemptId, Set<RMContainer>>();
-    List<RMContainer> skippedAMContainerlist = new ArrayList<RMContainer>();
+        new HashMap<>();
+    List<RMContainer> skippedAMContainerlist = new ArrayList<>();
 
     // Loop all leaf queues
     for (String queueName : leafQueueNames) {
@@ -614,7 +684,7 @@ public class ProportionalCapacityPreemptionPolicy implements SchedulingEditPolic
       LeafQueue leafQueue = null;
 
       Map<String, Resource> resToObtainByPartition =
-          new HashMap<String, Resource>();
+          new HashMap<>();
       for (TempQueuePerPartition qT : getQueuePartitions(queueName)) {
         leafQueue = qT.leafQueue;
         // we act only if we are violating balance by more than
@@ -703,7 +773,6 @@ public class ProportionalCapacityPreemptionPolicy implements SchedulingEditPolic
    * @param clusterResource
    * @param preemptMap
    * @param skippedAMContainerlist
-   * @param resToObtain
    * @param skippedAMSize
    * @param maxAMCapacityForThisQueue
    */
@@ -751,7 +820,7 @@ public class ProportionalCapacityPreemptionPolicy implements SchedulingEditPolic
 
     // first drop reserved containers towards rsrcPreempt
     List<RMContainer> reservedContainers =
-        new ArrayList<RMContainer>(app.getReservedContainers());
+        new ArrayList<>(app.getReservedContainers());
     for (RMContainer c : reservedContainers) {
       if (resToObtainByPartition.isEmpty()) {
         return;
@@ -771,8 +840,7 @@ public class ProportionalCapacityPreemptionPolicy implements SchedulingEditPolic
     // if more resources are to be freed go through all live containers in
     // reverse priority and reverse allocation order and mark them for
     // preemption
-    List<RMContainer> liveContainers =
-      new ArrayList<RMContainer>(app.getLiveContainers());
+    List<RMContainer> liveContainers = new ArrayList<>(app.getLiveContainers());
 
     sortContainers(liveContainers);
 
@@ -788,6 +856,11 @@ public class ProportionalCapacityPreemptionPolicy implements SchedulingEditPolic
         continue;
       }
 
+      // Skip already marked to killable containers
+      if (killableContainers.contains(c.getContainerId())) {
+        continue;
+      }
+
       // Try to preempt this container
       tryPreemptContainerAndDeductResToObtain(resToObtainByPartition, c,
           clusterResource, preemptMap);
@@ -826,6 +899,10 @@ public class ProportionalCapacityPreemptionPolicy implements SchedulingEditPolic
     return "ProportionalCapacityPreemptionPolicy";
   }
 
+  @VisibleForTesting
+  public Map<RMContainer, Long> getToPreemptContainers() {
+    return preempted;
+  }
 
   /**
    * This method walks a tree of CSQueue and clones the portion of the state
@@ -851,6 +928,11 @@ public class ProportionalCapacityPreemptionPolicy implements SchedulingEditPolic
           partitionToLookAt);
       Resource guaranteed = Resources.multiply(partitionResource, absCap);
       Resource maxCapacity = Resources.multiply(partitionResource, absMaxCap);
+      Resource killable = Resources.none();
+      if (null != preemptableEntities.get(queueName)) {
+         killable = preemptableEntities.get(queueName)
+            .getKillableResource(partitionToLookAt);
+      }
 
       // when partition is a non-exclusive partition, the actual maxCapacity
       // could more than specified maxCapacity
@@ -875,7 +957,7 @@ public class ProportionalCapacityPreemptionPolicy implements SchedulingEditPolic
               l.getTotalPendingResourcesConsideringUserLimit(
                   partitionResource, partitionToLookAt);
         ret = new TempQueuePerPartition(queueName, current, pending, guaranteed,
-            maxCapacity, preemptionDisabled, partitionToLookAt);
+            maxCapacity, preemptionDisabled, partitionToLookAt, killable);
         if (preemptionDisabled) {
           ret.untouchableExtra = extra;
         } else {
@@ -886,7 +968,7 @@ public class ProportionalCapacityPreemptionPolicy implements SchedulingEditPolic
         Resource pending = Resource.newInstance(0, 0);
         ret =
             new TempQueuePerPartition(curQueue.getQueueName(), current, pending,
-                guaranteed, maxCapacity, false, partitionToLookAt);
+                guaranteed, maxCapacity, false, partitionToLookAt, killable);
         Resource childrensPreemptable = Resource.newInstance(0, 0);
         for (CSQueue c : curQueue.getChildQueues()) {
           TempQueuePerPartition subq =
@@ -932,7 +1014,7 @@ public class ProportionalCapacityPreemptionPolicy implements SchedulingEditPolic
 
     Map<String, TempQueuePerPartition> queuePartitions;
     if (null == (queuePartitions = queueToPartitions.get(queueName))) {
-      queuePartitions = new HashMap<String, TempQueuePerPartition>();
+      queuePartitions = new HashMap<>();
       queueToPartitions.put(queueName, queuePartitions);
     }
     queuePartitions.put(queuePartition.partition, queuePartition);
@@ -971,8 +1053,10 @@ public class ProportionalCapacityPreemptionPolicy implements SchedulingEditPolic
     final Resource guaranteed;
     final Resource maxCapacity;
     final String partition;
+    final Resource killable;
     Resource idealAssigned;
     Resource toBePreempted;
+
     // For logging purpose
     Resource actuallyPreempted;
     Resource untouchableExtra;
@@ -986,7 +1070,7 @@ public class ProportionalCapacityPreemptionPolicy implements SchedulingEditPolic
 
     TempQueuePerPartition(String queueName, Resource current, Resource pending,
         Resource guaranteed, Resource maxCapacity, boolean preemptionDisabled,
-        String partition) {
+        String partition, Resource killableResource) {
       this.queueName = queueName;
       this.current = current;
       this.pending = pending;
@@ -996,11 +1080,12 @@ public class ProportionalCapacityPreemptionPolicy implements SchedulingEditPolic
       this.actuallyPreempted = Resource.newInstance(0, 0);
       this.toBePreempted = Resource.newInstance(0, 0);
       this.normalizedGuarantee = Float.NaN;
-      this.children = new ArrayList<TempQueuePerPartition>();
+      this.children = new ArrayList<>();
       this.untouchableExtra = Resource.newInstance(0, 0);
       this.preemptableExtra = Resource.newInstance(0, 0);
       this.preemptionDisabled = preemptionDisabled;
       this.partition = partition;
+      this.killable = killableResource;
     }
 
     public void setLeafQueue(LeafQueue l){
@@ -1018,12 +1103,6 @@ public class ProportionalCapacityPreemptionPolicy implements SchedulingEditPolic
       Resources.addTo(pending, q.pending);
     }
 
-    public void addChildren(ArrayList<TempQueuePerPartition> queues) {
-      assert leafQueue == null;
-      children.addAll(queues);
-    }
-
-
     public ArrayList<TempQueuePerPartition> getChildren(){
       return children;
     }
@@ -1064,18 +1143,13 @@ public class ProportionalCapacityPreemptionPolicy implements SchedulingEditPolic
       return sb.toString();
     }
 
-    public void printAll() {
-      LOG.info(this.toString());
-      for (TempQueuePerPartition sub : this.getChildren()) {
-        sub.printAll();
-      }
-    }
-
     public void assignPreemption(float scalingFactor,
         ResourceCalculator rc, Resource clusterResource) {
-      if (Resources.greaterThan(rc, clusterResource, current, idealAssigned)) {
-          toBePreempted = Resources.multiply(
-              Resources.subtract(current, idealAssigned), scalingFactor);
+      if (Resources.greaterThan(rc, clusterResource,
+          Resources.subtract(current, killable), idealAssigned)) {
+        toBePreempted = Resources.multiply(Resources.subtract(
+            Resources.subtract(current, killable), idealAssigned),
+            scalingFactor);
       } else {
         toBePreempted = Resource.newInstance(0, 0);
       }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7e8c9beb/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 5d26931..dfe0886 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
@@ -30,6 +30,7 @@ import org.apache.hadoop.yarn.api.records.Priority;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.ResourceRequest;
 import org.apache.hadoop.yarn.event.EventHandler;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Queue;
 
 /**
  * Represents the ResourceManager's view of an application container. See 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7e8c9beb/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/PreemptableResourceScheduler.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/PreemptableResourceScheduler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/PreemptableResourceScheduler.java
index ee7e101..b73c538 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/PreemptableResourceScheduler.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/PreemptableResourceScheduler.java
@@ -45,6 +45,6 @@ public interface PreemptableResourceScheduler extends ResourceScheduler {
    * Ask the scheduler to forcibly interrupt the container given as input
    * @param container
    */
-  void killPreemptedContainer(RMContainer container);
+  void markContainerForKillable(RMContainer container);
 
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7e8c9beb/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/ResourceLimits.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/ResourceLimits.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/ResourceLimits.java
index c545e9e..721eb36 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/ResourceLimits.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/ResourceLimits.java
@@ -38,6 +38,8 @@ public class ResourceLimits {
   // containers.
   private volatile Resource headroom;
 
+  private boolean allowPreempt = false;
+
   public ResourceLimits(Resource limit) {
     this(limit, Resources.none());
   }
@@ -72,4 +74,11 @@ public class ResourceLimits {
     this.amountNeededUnreserve = amountNeededUnreserve;
   }
 
+  public boolean isAllowPreemption() {
+    return allowPreempt;
+  }
+
+  public void setIsAllowPreemption(boolean allowPreempt) {
+   this.allowPreempt = allowPreempt;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7e8c9beb/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerNode.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/SchedulerNode.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerNode.java
index 33ab2f1..6c4f300 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerNode.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerNode.java
@@ -64,9 +64,8 @@ public abstract class SchedulerNode {
   private volatile ResourceUtilization nodeUtilization =
       ResourceUtilization.newInstance(0, 0, 0f);
 
-
-  /** Set of containers that are allocated containers. */
-  private final Map<ContainerId, RMContainer> launchedContainers =
+  /* set of containers that are allocated containers */
+  protected final Map<ContainerId, RMContainer> launchedContainers =
       new HashMap<>();
 
   private final RMNode rmNode;
@@ -168,7 +167,7 @@ public abstract class SchedulerNode {
    * @param deltaResource Change in the resource allocation.
    * @param increase True if the change is an increase of allocation.
    */
-  private synchronized void changeContainerResource(ContainerId containerId,
+  protected synchronized void changeContainerResource(ContainerId containerId,
       Resource deltaResource, boolean increase) {
     if (increase) {
       deductUnallocatedResource(deltaResource);
@@ -242,7 +241,7 @@ public abstract class SchedulerNode {
    * Update the resources of the node when allocating a new container.
    * @param container Container to allocate.
    */
-  private synchronized void updateResource(Container container) {
+  protected synchronized void updateResource(Container container) {
     addUnallocatedResource(container.getResource());
     --numContainers;
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7e8c9beb/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 39ca29b..955f8fa 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
@@ -21,6 +21,7 @@ package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity;
 import java.io.IOException;
 import java.util.HashMap;
 import java.util.HashSet;
+import java.util.Iterator;
 import java.util.Map;
 import java.util.Set;
 
@@ -45,6 +46,7 @@ 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.rmcontainer.RMContainer;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceLimits;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceUsage;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplicationAttempt;
@@ -440,11 +442,8 @@ public abstract class AbstractCSQueue implements CSQueue {
           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;  
+      return Resources.min(resourceCalculator, clusterResource,
+          queueMaxResource, currentResourceLimits.getLimit());
     } 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.
@@ -474,12 +473,19 @@ public abstract class AbstractCSQueue implements CSQueue {
 
     Resource nowTotalUsed = queueUsage.getUsed(nodePartition);
 
-    // Set headroom for currentResourceLimits
-    currentResourceLimits.setHeadroom(Resources.subtract(currentLimitResource,
-        nowTotalUsed));
+    // Set headroom for currentResourceLimits:
+    // When queue is a parent queue: Headroom = limit - used + killable
+    // When queue is a leaf queue: Headroom = limit - used (leaf queue cannot preempt itself)
+    Resource usedExceptKillable = nowTotalUsed;
+    if (null != getChildQueues() && !getChildQueues().isEmpty()) {
+      usedExceptKillable = Resources.subtract(nowTotalUsed,
+          getTotalKillableResource(nodePartition));
+    }
+    currentResourceLimits.setHeadroom(
+        Resources.subtract(currentLimitResource, usedExceptKillable));
 
     if (Resources.greaterThanOrEqual(resourceCalculator, clusterResource,
-        nowTotalUsed, currentLimitResource)) {
+        usedExceptKillable, currentLimitResource)) {
 
       // if reservation continous looking enabled, check to see if could we
       // potentially use this node instead of a reserved node if the application
@@ -491,7 +497,7 @@ public abstract class AbstractCSQueue implements CSQueue {
               resourceCouldBeUnreserved, Resources.none())) {
         // resource-without-reserved = used - reserved
         Resource newTotalWithoutReservedResource =
-            Resources.subtract(nowTotalUsed, resourceCouldBeUnreserved);
+            Resources.subtract(usedExceptKillable, resourceCouldBeUnreserved);
 
         // when total-used-without-reserved-resource < currentLimit, we still
         // have chance to allocate on this node by unreserving some containers
@@ -620,11 +626,10 @@ public abstract class AbstractCSQueue implements CSQueue {
     // considering all labels in cluster, only those labels which are
     // use some resource of this queue can be considered.
     Set<String> nodeLabels = new HashSet<String>();
-    if (this.getAccessibleNodeLabels() != null
-        && this.getAccessibleNodeLabels().contains(RMNodeLabelsManager.ANY)) {
-      nodeLabels.addAll(Sets.union(this.getQueueCapacities()
-          .getNodePartitionsSet(), this.getQueueResourceUsage()
-          .getNodePartitionsSet()));
+    if (this.getAccessibleNodeLabels() != null && this.getAccessibleNodeLabels()
+        .contains(RMNodeLabelsManager.ANY)) {
+      nodeLabels.addAll(Sets.union(this.getQueueCapacities().getNodePartitionsSet(),
+          this.getQueueResourceUsage().getNodePartitionsSet()));
     } else {
       nodeLabels.addAll(this.getAccessibleNodeLabels());
     }
@@ -636,4 +641,14 @@ public abstract class AbstractCSQueue implements CSQueue {
     }
     return nodeLabels;
   }
+
+  public Resource getTotalKillableResource(String partition) {
+    return csContext.getPreemptionManager().getKillableResource(queueName,
+        partition);
+  }
+
+  public Iterator<RMContainer> getKillableContainers(String partition) {
+    return csContext.getPreemptionManager().getKillableContainers(queueName,
+        partition);
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7e8c9beb/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CSAssignment.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/CSAssignment.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/CSAssignment.java
index 68f6f12..6406efe 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/CSAssignment.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/CSAssignment.java
@@ -26,6 +26,8 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.Assignment
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerApp;
 import org.apache.hadoop.yarn.util.resource.Resources;
 
+import java.util.List;
+
 @Private
 @Unstable
 public class CSAssignment {
@@ -42,6 +44,7 @@ public class CSAssignment {
   private boolean fulfilledReservation;
   private final AssignmentInformation assignmentInformation;
   private boolean increaseAllocation;
+  private List<RMContainer> containersToKill;
 
   public CSAssignment(Resource resource, NodeType type) {
     this(resource, type, null, null, false, false);
@@ -147,4 +150,12 @@ public class CSAssignment {
   public void setIncreasedAllocation(boolean flag) {
     increaseAllocation = flag;
   }
+
+  public void setContainersToKill(List<RMContainer> containersToKill) {
+    this.containersToKill = containersToKill;
+  }
+
+  public List<RMContainer> getContainersToKill() {
+    return containersToKill;
+  }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7e8c9beb/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 735306a..cf5c3b5 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
@@ -108,6 +108,8 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerDynamicE
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerHealth;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNode;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerUtils;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.preemption.KillableContainer;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.preemption.PreemptionManager;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.AssignmentInformation;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.QueueEntitlement;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerApp;
@@ -148,6 +150,10 @@ public class CapacityScheduler extends
   // timeout to join when we stop this service
   protected final long THREAD_JOIN_TIMEOUT_MS = 1000;
 
+  private PreemptionManager preemptionManager = new PreemptionManager();
+
+  private volatile boolean isLazyPreemptionEnabled = false;
+
   static final Comparator<CSQueue> nonPartitionedQueueComparator =
       new Comparator<CSQueue>() {
     @Override
@@ -298,12 +304,11 @@ public class CapacityScheduler extends
     initMaximumResourceCapability(this.conf.getMaximumAllocation());
     this.calculator = this.conf.getResourceCalculator();
     this.usePortForNodeName = this.conf.getUsePortForNodeName();
-    this.applications =
-        new ConcurrentHashMap<ApplicationId,
-            SchedulerApplication<FiCaSchedulerApp>>();
+    this.applications = new ConcurrentHashMap<>();
     this.labelManager = rmContext.getNodeLabelManager();
     authorizer = YarnAuthorizationProvider.getInstance(yarnConf);
     initializeQueues(this.conf);
+    this.isLazyPreemptionEnabled = conf.getLazyPreemptionEnabled();
 
     scheduleAsynchronously = this.conf.getScheduleAynschronously();
     asyncScheduleInterval =
@@ -369,6 +374,9 @@ public class CapacityScheduler extends
       refreshMaximumAllocation(this.conf.getMaximumAllocation());
       throw new IOException("Failed to re-init queues", t);
     }
+
+    // update lazy preemption
+    this.isLazyPreemptionEnabled = this.conf.getLazyPreemptionEnabled();
   }
   
   long getAsyncScheduleInterval() {
@@ -503,6 +511,9 @@ public class CapacityScheduler extends
     LOG.info("Initialized root queue " + root);
     updatePlacementRules();
     setQueueAcls(authorizer, queues);
+
+    // Notify Preemption Manager
+    preemptionManager.refreshQueues(null, root);
   }
 
   @Lock(CapacityScheduler.class)
@@ -531,6 +542,9 @@ public class CapacityScheduler extends
 
     labelManager.reinitializeQueueLabels(getQueueToLabels());
     setQueueAcls(authorizer, queues);
+
+    // Notify Preemption Manager
+    preemptionManager.refreshQueues(null, root);
   }
 
   @VisibleForTesting
@@ -1253,8 +1267,10 @@ public class CapacityScheduler extends
 
     // Try to schedule more if there are no reservations to fulfill
     if (node.getReservedContainer() == null) {
-      if (calculator.computeAvailableContainers(node.getUnallocatedResource(),
-        minimumAllocation) > 0) {
+      if (calculator.computeAvailableContainers(Resources
+              .add(node.getUnallocatedResource(), node.getTotalKillableResources()),
+          minimumAllocation) > 0) {
+
         if (LOG.isDebugEnabled()) {
           LOG.debug("Trying to schedule on node: " + node.getNodeName() +
               ", available: " + node.getUnallocatedResource());
@@ -1263,10 +1279,8 @@ public class CapacityScheduler extends
         assignment = root.assignContainers(
             getClusterResource(),
             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, getClusterResource())),
+                node.getPartition(), getClusterResource())),
             SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
         if (Resources.greaterThan(calculator, getClusterResource(),
             assignment.getResource(), Resources.none())) {
@@ -1436,11 +1450,20 @@ public class CapacityScheduler extends
       markContainerForPreemption(aid, containerToBePreempted);
     }
     break;
-    case KILL_PREEMPTED_CONTAINER:
+    case MARK_CONTAINER_FOR_KILLABLE:
+    {
+      ContainerPreemptEvent containerKillableEvent = (ContainerPreemptEvent)event;
+      RMContainer killableContainer = containerKillableEvent.getContainer();
+      markContainerForKillable(killableContainer);
+    }
+    break;
+    case MARK_CONTAINER_FOR_NONKILLABLE:
     {
-      ContainerPreemptEvent killContainerEvent = (ContainerPreemptEvent)event;
-      RMContainer containerToBeKilled = killContainerEvent.getContainer();
-      killPreemptedContainer(containerToBeKilled);
+      if (isLazyPreemptionEnabled) {
+        ContainerPreemptEvent cancelKillContainerEvent =
+            (ContainerPreemptEvent) event;
+        markContainerForNonKillable(cancelKillContainerEvent.getContainer());
+      }
     }
     break;
     default:
@@ -1548,14 +1571,14 @@ public class CapacityScheduler extends
   protected void completedContainerInternal(
       RMContainer rmContainer, ContainerStatus containerStatus,
       RMContainerEventType event) {
-    
     Container container = rmContainer.getContainer();
+    ContainerId containerId = container.getId();
     
     // Get the application for the finished container
     FiCaSchedulerApp application =
         getCurrentAttemptForContainer(container.getId());
     ApplicationId appId =
-        container.getId().getApplicationAttemptId().getApplicationId();
+        containerId.getApplicationAttemptId().getApplicationId();
     if (application == null) {
       LOG.info("Container " + container + " of" + " finished application "
           + appId + " completed with event " + event);
@@ -1569,15 +1592,6 @@ public class CapacityScheduler extends
     LeafQueue queue = (LeafQueue)application.getQueue();
     queue.completedContainer(getClusterResource(), application, node,
         rmContainer, containerStatus, event, null, true);
-
-    if (containerStatus.getExitStatus() == ContainerExitStatus.PREEMPTED) {
-      schedulerHealth.updatePreemption(Time.now(), container.getNodeId(),
-        container.getId(), queue.getQueuePath());
-      schedulerHealth.updateSchedulerPreemptionCounts(1);
-    } else {
-      schedulerHealth.updateRelease(lastNodeUpdateTime, container.getNodeId(),
-        container.getId(), queue.getQueuePath());
-    }
   }
   
   @Override
@@ -1613,7 +1627,7 @@ public class CapacityScheduler extends
       ApplicationAttemptId applicationAttemptId) {
     return super.getApplicationAttempt(applicationAttemptId);
   }
-  
+
   @Lock(Lock.NoLock.class)
   public FiCaSchedulerNode getNode(NodeId nodeId) {
     return nodeTracker.getNode(nodeId);
@@ -1654,15 +1668,60 @@ public class CapacityScheduler extends
     }
   }
 
-  @Override
-  public void killPreemptedContainer(RMContainer cont) {
+  public synchronized void markContainerForKillable(
+      RMContainer killableContainer) {
     if (LOG.isDebugEnabled()) {
-      LOG.debug(SchedulerEventType.KILL_PREEMPTED_CONTAINER + ": container"
-          + cont.toString());
+      LOG.debug(SchedulerEventType.MARK_CONTAINER_FOR_KILLABLE + ": container"
+          + killableContainer.toString());
+    }
+
+    if (!isLazyPreemptionEnabled) {
+      super.completedContainer(killableContainer, SchedulerUtils
+          .createPreemptedContainerStatus(killableContainer.getContainerId(),
+              SchedulerUtils.PREEMPTED_CONTAINER), RMContainerEventType.KILL);
+    } else {
+      FiCaSchedulerNode node = (FiCaSchedulerNode) getSchedulerNode(
+          killableContainer.getAllocatedNode());
+
+      FiCaSchedulerApp application = getCurrentAttemptForContainer(
+          killableContainer.getContainerId());
+
+      node.markContainerToKillable(killableContainer.getContainerId());
+
+      // notify PreemptionManager
+      // Get the application for the finished container
+      if (null != application) {
+        String leafQueueName = application.getCSLeafQueue().getQueueName();
+        getPreemptionManager().addKillableContainer(
+            new KillableContainer(killableContainer, node.getPartition(),
+                leafQueueName));
+      }    }
+  }
+
+  private synchronized void markContainerForNonKillable(
+      RMContainer nonKillableContainer) {
+    if (LOG.isDebugEnabled()) {
+      LOG.debug(
+          SchedulerEventType.MARK_CONTAINER_FOR_NONKILLABLE + ": container"
+              + nonKillableContainer.toString());
+    }
+
+    FiCaSchedulerNode node = (FiCaSchedulerNode) getSchedulerNode(
+        nonKillableContainer.getAllocatedNode());
+
+    FiCaSchedulerApp application = getCurrentAttemptForContainer(
+        nonKillableContainer.getContainerId());
+
+    node.markContainerToNonKillable(nonKillableContainer.getContainerId());
+
+    // notify PreemptionManager
+    // Get the application for the finished container
+    if (null != application) {
+      String leafQueueName = application.getCSLeafQueue().getQueueName();
+      getPreemptionManager().removeKillableContainer(
+          new KillableContainer(nonKillableContainer, node.getPartition(),
+              leafQueueName));
     }
-    super.completedContainer(cont, SchedulerUtils
-        .createPreemptedContainerStatus(cont.getContainerId(),
-        SchedulerUtils.PREEMPTED_CONTAINER), RMContainerEventType.KILL);
   }
 
   @Override
@@ -1945,6 +2004,7 @@ public class CapacityScheduler extends
     return ret;
   }
 
+  @Override
   public SchedulerHealth getSchedulerHealth() {
     return this.schedulerHealth;
   }
@@ -1954,6 +2014,11 @@ public class CapacityScheduler extends
   }
 
   @Override
+  public long getLastNodeUpdateTime() {
+    return lastNodeUpdateTime;
+  }
+
+  @Override
   public Priority checkAndGetApplicationPriority(Priority priorityFromContext,
       String user, String queueName, ApplicationId applicationId)
       throws YarnException {
@@ -2054,4 +2119,9 @@ public class CapacityScheduler extends
         + rmApp.getQueue() + " for application: " + applicationId
         + " for the user: " + rmApp.getUser());
   }
+
+  @Override
+  public PreemptionManager getPreemptionManager() {
+    return preemptionManager;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7e8c9beb/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 3756d9e..3729264 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
@@ -257,6 +257,12 @@ public class CapacitySchedulerConfiguration extends ReservationSchedulerConfigur
   public static final String RESERVATION_ENFORCEMENT_WINDOW =
       "reservation-enforcement-window";
 
+  @Private
+  public static final String LAZY_PREEMPTION_ENALBED = PREFIX + "lazy-preemption-enabled";
+
+  @Private
+  public static final boolean DEFAULT_LAZY_PREEMPTION_ENABLED = false;
+
   public CapacitySchedulerConfiguration() {
     this(new Configuration());
   }
@@ -1007,7 +1013,11 @@ public class CapacitySchedulerConfiguration extends ReservationSchedulerConfigur
   @VisibleForTesting
   public void setOrderingPolicyParameter(String queue,
       String parameterKey, String parameterValue) {
-    set(getQueuePrefix(queue) + ORDERING_POLICY + "."
-        + parameterKey, parameterValue);
+    set(getQueuePrefix(queue) + ORDERING_POLICY + "." + parameterKey,
+        parameterValue);
+  }
+
+  public boolean getLazyPreemptionEnabled() {
+    return getBoolean(LAZY_PREEMPTION_ENALBED, DEFAULT_LAZY_PREEMPTION_ENABLED);
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7e8c9beb/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 2a0dd0d..1203272 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
@@ -18,17 +18,20 @@
 
 package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity;
 
-import java.util.Comparator;
-
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerHealth;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.preemption.PreemptionManager;
 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.server.resourcemanager.security.RMContainerTokenSecretManager;
 import org.apache.hadoop.yarn.util.resource.ResourceCalculator;
 
+import java.util.Comparator;
+
 /**
  * Read-only interface to {@link CapacityScheduler} context.
  */
@@ -61,4 +64,12 @@ public interface CapacitySchedulerContext {
   PartitionedQueueComparator getPartitionedQueueComparator();
   
   FiCaSchedulerNode getNode(NodeId nodeId);
+
+  FiCaSchedulerApp getApplicationAttempt(ApplicationAttemptId attemptId);
+
+  PreemptionManager getPreemptionManager();
+
+  SchedulerHealth getSchedulerHealth();
+
+  long getLastNodeUpdateTime();
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7e8c9beb/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 c625fae..3dc2090 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
@@ -37,9 +37,11 @@ import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.security.AccessControlException;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.authorize.AccessControlList;
+import org.apache.hadoop.util.Time;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.Container;
+import org.apache.hadoop.yarn.api.records.ContainerExitStatus;
 import org.apache.hadoop.yarn.api.records.ContainerStatus;
 import org.apache.hadoop.yarn.api.records.Priority;
 import org.apache.hadoop.yarn.api.records.QueueACL;
@@ -63,7 +65,9 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceUsage;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedContainerChangeRequest;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplicationAttempt;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplicationAttempt.AMState;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerHealth;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerUtils;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.preemption.KillableContainer;
 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.server.resourcemanager.scheduler.policy.FifoOrderingPolicyForPendingApps;
@@ -823,6 +827,40 @@ public class LeafQueue extends AbstractCSQueue {
       assignment.setExcessReservation(null);
     }
   }
+
+  private void killToPreemptContainers(Resource clusterResource,
+      FiCaSchedulerNode node,
+      CSAssignment assignment) {
+    if (assignment.getContainersToKill() != null) {
+      StringBuilder sb = new StringBuilder("Killing containers: [");
+
+      for (RMContainer c : assignment.getContainersToKill()) {
+        FiCaSchedulerApp application = csContext.getApplicationAttempt(
+            c.getApplicationAttemptId());
+        LeafQueue q = application.getCSLeafQueue();
+        q.completedContainer(clusterResource, application, node, c, SchedulerUtils
+                .createPreemptedContainerStatus(c.getContainerId(),
+                    SchedulerUtils.PREEMPTED_CONTAINER), RMContainerEventType.KILL,
+            null, false);
+        sb.append("(container=" + c.getContainerId() + " resource=" + c
+            .getAllocatedResource() + ")");
+      }
+
+      sb.append("] for container=" + assignment.getAssignmentInformation()
+          .getFirstAllocatedOrReservedContainerId() + " resource=" + assignment
+          .getResource());
+      LOG.info(sb.toString());
+
+    }
+  }
+
+  private void setPreemptionAllowed(ResourceLimits limits, String nodePartition) {
+    // Set preemption-allowed:
+    // For leaf queue, only under-utilized queue is allowed to preempt resources from other queues
+    float usedCapacity = queueCapacities.getAbsoluteUsedCapacity(nodePartition);
+    float guaranteedCapacity = queueCapacities.getAbsoluteCapacity(nodePartition);
+    limits.setIsAllowPreemption(usedCapacity < guaranteedCapacity);
+  }
   
   @Override
   public synchronized CSAssignment assignContainers(Resource clusterResource,
@@ -835,6 +873,8 @@ public class LeafQueue extends AbstractCSQueue {
           + " #applications=" + orderingPolicy.getNumSchedulableEntities());
     }
 
+    setPreemptionAllowed(currentResourceLimits, node.getPartition());
+
     // Check for reserved resources
     RMContainer reservedContainer = node.getReservedContainer();
     if (reservedContainer != null) {
@@ -846,6 +886,7 @@ public class LeafQueue extends AbstractCSQueue {
                 currentResourceLimits, schedulingMode, reservedContainer);
         handleExcessReservedContainer(clusterResource, assignment, node,
             application);
+        killToPreemptContainers(clusterResource, node, assignment);
         return assignment;
       }
     }
@@ -907,6 +948,7 @@ public class LeafQueue extends AbstractCSQueue {
       
       handleExcessReservedContainer(clusterResource, assignment, node,
           application);
+      killToPreemptContainers(clusterResource, node, assignment);
 
       if (Resources.greaterThan(resourceCalculator, clusterResource, assigned,
           Resources.none())) {
@@ -1210,11 +1252,34 @@ public class LeafQueue extends AbstractCSQueue {
     }
   }
 
+  private void updateSchedulerHealthForCompletedContainer(
+      RMContainer rmContainer, ContainerStatus containerStatus) {
+    // Update SchedulerHealth for released / preempted container
+    SchedulerHealth schedulerHealth = csContext.getSchedulerHealth();
+    if (null == schedulerHealth) {
+      // Only do update if we have schedulerHealth
+      return;
+    }
+
+    if (containerStatus.getExitStatus() == ContainerExitStatus.PREEMPTED) {
+      schedulerHealth.updatePreemption(Time.now(), rmContainer.getAllocatedNode(),
+          rmContainer.getContainerId(), getQueuePath());
+      schedulerHealth.updateSchedulerPreemptionCounts(1);
+    } else {
+      schedulerHealth.updateRelease(csContext.getLastNodeUpdateTime(),
+          rmContainer.getAllocatedNode(), rmContainer.getContainerId(),
+          getQueuePath());
+    }
+  }
+
   @Override
   public void completedContainer(Resource clusterResource, 
       FiCaSchedulerApp application, FiCaSchedulerNode node, RMContainer rmContainer, 
       ContainerStatus containerStatus, RMContainerEventType event, CSQueue childQueue,
       boolean sortQueues) {
+    // Update SchedulerHealth for released / preempted container
+    updateSchedulerHealthForCompletedContainer(rmContainer, containerStatus);
+
     if (application != null) {
       // unreserve container increase request if it previously reserved.
       if (rmContainer.hasIncreaseReservation()) {
@@ -1265,6 +1330,10 @@ public class LeafQueue extends AbstractCSQueue {
           rmContainer, null, event, this, sortQueues);
       }
     }
+
+    // Notify PreemptionManager
+    csContext.getPreemptionManager().removeKillableContainer(
+        new KillableContainer(rmContainer, node.getPartition(), queueName));
   }
 
   synchronized void allocateResource(Resource clusterResource,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7e8c9beb/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 7cf5565..6fcd6c1 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
@@ -18,18 +18,6 @@
 
 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;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.TreeSet;
-
 import org.apache.commons.lang.StringUtils;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -49,6 +37,7 @@ import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.exceptions.InvalidResourceRequestException;
 import org.apache.hadoop.yarn.factories.RecordFactory;
 import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
+import org.apache.hadoop.yarn.nodelabels.RMNodeLabel;
 import org.apache.hadoop.yarn.security.AccessType;
 import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager;
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
@@ -57,12 +46,25 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerStat
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ActiveUsersManager;
 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.SchedContainerChangeRequest;
+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.fica.FiCaSchedulerApp;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerNode;
 import org.apache.hadoop.yarn.util.resource.Resources;
 
+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;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.TreeSet;
+
 @Private
 @Evolving
 public class ParentQueue extends AbstractCSQueue {
@@ -386,6 +388,11 @@ public class ParentQueue extends AbstractCSQueue {
     // if our queue cannot access this node, just return
     if (schedulingMode == SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY
         && !accessibleToPartition(node.getPartition())) {
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Skip this queue=" + getQueuePath()
+            + ", because it is not able to access partition=" + node
+            .getPartition());
+      }
       return CSAssignment.NULL_ASSIGNMENT;
     }
     
@@ -431,7 +438,7 @@ public class ParentQueue extends AbstractCSQueue {
               resourceCalculator, clusterResource, 
               assignedToChild.getResource(), Resources.none())) {
         // Track resource utilization for the parent-queue
-        super.allocateResource(clusterResource, assignedToChild.getResource(),
+        allocateResource(clusterResource, assignedToChild.getResource(),
             node.getPartition(), assignedToChild.isIncreasedAllocation());
         
         // Track resource utilization in this pass of the scheduler
@@ -494,29 +501,38 @@ public class ParentQueue extends AbstractCSQueue {
   }
 
   private boolean canAssign(Resource clusterResource, FiCaSchedulerNode node) {
-    return (node.getReservedContainer() == null) && 
-        Resources.greaterThanOrEqual(resourceCalculator, clusterResource, 
-            node.getUnallocatedResource(), minimumAllocation);
+    // Two conditions need to meet when trying to allocate:
+    // 1) Node doesn't have reserved container
+    // 2) Node's available-resource + killable-resource should > 0
+    return node.getReservedContainer() == null && Resources.greaterThanOrEqual(
+        resourceCalculator, clusterResource, Resources
+            .add(node.getUnallocatedResource(), node.getTotalKillableResources()),
+        minimumAllocation);
   }
-  
+
   private ResourceLimits getResourceLimitsOfChild(CSQueue child,
-      Resource clusterResource, ResourceLimits parentLimits) {
+      Resource clusterResource, ResourceLimits parentLimits,
+      String nodePartition) {
     // Set resource-limit of a given child, child.limit =
     // min(my.limit - my.used + child.used, child.max)
 
     // Parent available resource = parent-limit - parent-used-resource
-    Resource parentMaxAvailableResource =
-        Resources.subtract(parentLimits.getLimit(), getUsedResources());
+    Resource parentMaxAvailableResource = Resources.subtract(
+        parentLimits.getLimit(), queueUsage.getUsed(nodePartition));
+    // Deduct killable from used
+    Resources.addTo(parentMaxAvailableResource,
+        getTotalKillableResource(nodePartition));
 
     // Child's limit = parent-available-resource + child-used
-    Resource childLimit =
-        Resources.add(parentMaxAvailableResource, child.getUsedResources());
+    Resource childLimit = Resources.add(parentMaxAvailableResource,
+        child.getQueueResourceUsage().getUsed(nodePartition));
 
     // Get child's max resource
-    Resource childConfiguredMaxResource =
-        Resources.multiplyAndNormalizeDown(resourceCalculator, labelManager
-            .getResourceByLabel(RMNodeLabelsManager.NO_LABEL, clusterResource),
-            child.getAbsoluteMaximumCapacity(), minimumAllocation);
+    Resource childConfiguredMaxResource = Resources.multiplyAndNormalizeDown(
+        resourceCalculator,
+        labelManager.getResourceByLabel(nodePartition, clusterResource),
+        child.getQueueCapacities().getAbsoluteMaximumCapacity(nodePartition),
+        minimumAllocation);
 
     // Child's limit should be capped by child configured max resource
     childLimit =
@@ -568,7 +584,7 @@ public class ParentQueue extends AbstractCSQueue {
 
       // Get ResourceLimits of child queue before assign containers
       ResourceLimits childLimits =
-          getResourceLimitsOfChild(childQueue, cluster, limits);
+          getResourceLimitsOfChild(childQueue, cluster, limits, node.getPartition());
       
       assignment = childQueue.assignContainers(cluster, node, 
           childLimits, schedulingMode);
@@ -714,8 +730,8 @@ public class ParentQueue extends AbstractCSQueue {
     // Update all children
     for (CSQueue childQueue : childQueues) {
       // Get ResourceLimits of child queue before assign containers
-      ResourceLimits childLimits =
-          getResourceLimitsOfChild(childQueue, clusterResource, resourceLimits);     
+      ResourceLimits childLimits = getResourceLimitsOfChild(childQueue,
+          clusterResource, resourceLimits, RMNodeLabelsManager.NO_LABEL);
       childQueue.updateClusterResource(clusterResource, childLimits);
     }
     
@@ -738,8 +754,8 @@ public class ParentQueue extends AbstractCSQueue {
     synchronized (this) {
       FiCaSchedulerNode node =
           scheduler.getNode(rmContainer.getContainer().getNodeId());
-      super.allocateResource(clusterResource, rmContainer.getContainer()
-          .getResource(), node.getPartition(), false);
+      allocateResource(clusterResource,
+          rmContainer.getContainer().getResource(), node.getPartition(), false);
     }
     if (parent != null) {
       parent.recoverContainer(clusterResource, attempt, rmContainer);
@@ -766,7 +782,7 @@ public class ParentQueue extends AbstractCSQueue {
     if (application != null) {
       FiCaSchedulerNode node =
           scheduler.getNode(rmContainer.getContainer().getNodeId());
-      super.allocateResource(clusterResource, rmContainer.getContainer()
+      allocateResource(clusterResource, rmContainer.getContainer()
           .getResource(), node.getPartition(), false);
       LOG.info("movedContainer" + " queueMoveIn=" + getQueueName()
           + " usedCapacity=" + getUsedCapacity() + " absoluteUsedCapacity="
@@ -802,4 +818,79 @@ public class ParentQueue extends AbstractCSQueue {
   public synchronized int getNumApplications() {
     return numApplications;
   }
+
+  synchronized void allocateResource(Resource clusterResource,
+      Resource resource, String nodePartition, boolean changeContainerResource) {
+    super.allocateResource(clusterResource, resource, nodePartition,
+        changeContainerResource);
+
+    /**
+     * check if we need to kill (killable) containers if maximum resource violated.
+     * Doing this because we will deduct killable resource when going from root.
+     * For example:
+     * <pre>
+     *      Root
+     *      /   \
+     *     a     b
+     *   /  \
+     *  a1  a2
+     * </pre>
+     *
+     * a: max=10G, used=10G, killable=2G
+     * a1: used=8G, killable=2G
+     * a2: used=2G, pending=2G, killable=0G
+     *
+     * When we get queue-a to allocate resource, even if queue-a
+     * reaches its max resource, we deduct its used by killable, so we can allocate
+     * at most 2G resources. ResourceLimits passed down to a2 has headroom set to 2G.
+     *
+     * If scheduler finds a 2G available resource in existing cluster, and assigns it
+     * to a2, now a2's used= 2G + 2G = 4G, and a's used = 8G + 4G = 12G > 10G
+     *
+     * When this happens, we have to preempt killable container (on same or different
+     * nodes) of parent queue to avoid violating parent's max resource.
+     */
+    if (getQueueCapacities().getAbsoluteMaximumCapacity(nodePartition)
+        < getQueueCapacities().getAbsoluteUsedCapacity(nodePartition)) {
+      killContainersToEnforceMaxQueueCapacity(nodePartition, clusterResource);
+    }
+  }
+
+  private void killContainersToEnforceMaxQueueCapacity(String partition,
+      Resource clusterResource) {
+    Iterator<RMContainer> killableContainerIter = getKillableContainers(
+        partition);
+    if (!killableContainerIter.hasNext()) {
+      return;
+    }
+
+    Resource partitionResource = labelManager.getResourceByLabel(partition,
+        null);
+    Resource maxResource = Resources.multiply(partitionResource,
+        getQueueCapacities().getAbsoluteMaximumCapacity(partition));
+
+    while (Resources.greaterThan(resourceCalculator, partitionResource,
+        queueUsage.getUsed(partition), maxResource)) {
+      RMContainer toKillContainer = killableContainerIter.next();
+      FiCaSchedulerApp attempt = csContext.getApplicationAttempt(
+          toKillContainer.getContainerId().getApplicationAttemptId());
+      FiCaSchedulerNode node = csContext.getNode(
+          toKillContainer.getAllocatedNode());
+      if (null != attempt && null != node) {
+        LeafQueue lq = attempt.getCSLeafQueue();
+        lq.completedContainer(clusterResource, attempt, node, toKillContainer,
+            SchedulerUtils.createPreemptedContainerStatus(
+                toKillContainer.getContainerId(),
+                SchedulerUtils.PREEMPTED_CONTAINER), RMContainerEventType.KILL,
+            null, false);
+        LOG.info("Killed container=" + toKillContainer.getContainerId()
+            + " from queue=" + lq.getQueueName() + " to make queue=" + this
+            .getQueueName() + "'s max-capacity enforced");
+      }
+
+      if (!killableContainerIter.hasNext()) {
+        break;
+      }
+    }
+  }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7e8c9beb/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/allocator/AbstractContainerAllocator.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/allocator/AbstractContainerAllocator.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/allocator/AbstractContainerAllocator.java
index ee01bd1..afac235 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/allocator/AbstractContainerAllocator.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/allocator/AbstractContainerAllocator.java
@@ -108,6 +108,8 @@ public abstract class AbstractContainerAllocator {
           assignment.setFulfilledReservation(true);
         }
       }
+
+      assignment.setContainersToKill(result.getToKillContainers());
     }
     
     return assignment;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7e8c9beb/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/allocator/ContainerAllocation.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/allocator/ContainerAllocation.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/allocator/ContainerAllocation.java
index 1df9410..8f749f6 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/allocator/ContainerAllocation.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/allocator/ContainerAllocation.java
@@ -19,11 +19,14 @@
 package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.allocator;
 
 import org.apache.hadoop.yarn.api.records.Container;
+import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.NodeType;
 import org.apache.hadoop.yarn.util.resource.Resources;
 
+import java.util.List;
+
 public class ContainerAllocation {
   /**
    * Skip the locality (e.g. node-local, rack-local, any), and look at other
@@ -56,6 +59,7 @@ public class ContainerAllocation {
   NodeType containerNodeType = NodeType.NODE_LOCAL;
   NodeType requestNodeType = NodeType.NODE_LOCAL;
   Container updatedContainer;
+  private List<RMContainer> toKillContainers;
 
   public ContainerAllocation(RMContainer containerToBeUnreserved,
       Resource resourceToBeAllocated, AllocationState state) {
@@ -86,4 +90,12 @@ public class ContainerAllocation {
   public Container getUpdatedContainer() {
     return updatedContainer;
   }
+
+  public void setToKillContainers(List<RMContainer> toKillContainers) {
+    this.toKillContainers = toKillContainers;
+  }
+
+  public List<RMContainer> getToKillContainers() {
+    return toKillContainers;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7e8c9beb/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/allocator/RegularContainerAllocator.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/allocator/RegularContainerAllocator.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/allocator/RegularContainerAllocator.java
index e168edf..a5ca2d8 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/allocator/RegularContainerAllocator.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/allocator/RegularContainerAllocator.java
@@ -42,6 +42,9 @@ import org.apache.hadoop.yarn.server.utils.BuilderUtils;
 import org.apache.hadoop.yarn.util.resource.ResourceCalculator;
 import org.apache.hadoop.yarn.util.resource.Resources;
 
+import java.util.ArrayList;
+import java.util.List;
+
 /**
  * Allocate normal (new) containers, considers locality/label, etc. Using
  * delayed scheduling mechanism to get better locality allocation.
@@ -435,9 +438,6 @@ public class RegularContainerAllocator extends AbstractContainerAllocator {
       return ContainerAllocation.LOCALITY_SKIPPED;
     }
 
-    assert Resources.greaterThan(
-        rc, clusterResource, available, Resources.none());
-
     boolean shouldAllocOrReserveNewContainer = shouldAllocOrReserveNewContainer(
         priority, capability);
 
@@ -460,6 +460,29 @@ public class RegularContainerAllocator extends AbstractContainerAllocator {
     boolean reservationsContinueLooking =
         application.getCSLeafQueue().getReservationContinueLooking();
 
+    // Check if we need to kill some containers to allocate this one
+    List<RMContainer> toKillContainers = null;
+    if (availableContainers == 0 && currentResoureLimits.isAllowPreemption()) {
+      Resource availableAndKillable = Resources.clone(available);
+      for (RMContainer killableContainer : node
+          .getKillableContainers().values()) {
+        if (null == toKillContainers) {
+          toKillContainers = new ArrayList<>();
+        }
+        toKillContainers.add(killableContainer);
+        Resources.addTo(availableAndKillable,
+                        killableContainer.getAllocatedResource());
+        if (Resources.fitsIn(rc,
+                             clusterResource,
+                             capability,
+                             availableAndKillable)) {
+          // Stop if we find enough spaces
+          availableContainers = 1;
+          break;
+        }
+      }
+    }
+
     if (availableContainers > 0) {
       // Allocate...
       // We will only do continuous reservation when this is not allocated from
@@ -499,12 +522,12 @@ public class RegularContainerAllocator extends AbstractContainerAllocator {
           new ContainerAllocation(unreservedContainer, request.getCapability(),
               AllocationState.ALLOCATED);
       result.containerNodeType = type;
+      result.setToKillContainers(toKillContainers);
       return result;
     } else {
       // if we are allowed to allocate but this node doesn't have space, reserve
       // it or if this was an already a reserved container, reserve it again
       if (shouldAllocOrReserveNewContainer || rmContainer != null) {
-
         if (reservationsContinueLooking && rmContainer == null) {
           // we could possibly ignoring queue capacity or user limits when
           // reservationsContinueLooking is set. Make sure we didn't need to
@@ -522,6 +545,7 @@ public class RegularContainerAllocator extends AbstractContainerAllocator {
             new ContainerAllocation(null, request.getCapability(),
                 AllocationState.RESERVED);
         result.containerNodeType = type;
+        result.setToKillContainers(null);
         return result;
       }
       // Skip the locality request
@@ -613,8 +637,7 @@ public class RegularContainerAllocator extends AbstractContainerAllocator {
   }
 
   ContainerAllocation doAllocation(ContainerAllocation allocationResult,
-      Resource clusterResource, FiCaSchedulerNode node,
-      SchedulingMode schedulingMode, Priority priority,
+      FiCaSchedulerNode node, Priority priority,
       RMContainer reservedContainer) {
     // Create the container if necessary
     Container container =
@@ -678,9 +701,7 @@ public class RegularContainerAllocator extends AbstractContainerAllocator {
 
     if (AllocationState.ALLOCATED == result.state
         || AllocationState.RESERVED == result.state) {
-      result =
-          doAllocation(result, clusterResource, node, schedulingMode, priority,
-              reservedContainer);
+      result = doAllocation(result, node, priority, reservedContainer);
     }
 
     return result;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7e8c9beb/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/preemption/KillableContainer.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/preemption/KillableContainer.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/preemption/KillableContainer.java
new file mode 100644
index 0000000..675b0b4
--- /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/preemption/KillableContainer.java
@@ -0,0 +1,45 @@
+/**
+ * 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
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
+ * 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.preemption;
+
+import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
+
+public class KillableContainer {
+  RMContainer container;
+  String partition;
+  String leafQueueName;
+
+  public KillableContainer(RMContainer container, String partition, String leafQueueName) {
+    this.container = container;
+    this.partition = partition;
+    this.leafQueueName = leafQueueName;
+  }
+
+  public RMContainer getRMContainer() {
+    return this.container;
+  }
+
+  public String getNodePartition() {
+    return this.partition;
+  }
+
+  public String getLeafQueueName() {
+    return this.leafQueueName;
+  }
+}


[10/46] hadoop git commit: HDFS-9941. Do not log StandbyException on NN, other minor logging fixes. Contributed by Arpit Agarwal.

Posted by ae...@apache.org.
HDFS-9941. Do not log StandbyException on NN, other minor logging fixes. Contributed by Arpit Agarwal.


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

Branch: refs/heads/HDFS-7240
Commit: 5644137adad30c84e40d2c4719627b3aabc73628
Parents: f291d82
Author: Chris Nauroth <cn...@apache.org>
Authored: Mon Mar 14 09:54:54 2016 -0700
Committer: Chris Nauroth <cn...@apache.org>
Committed: Mon Mar 14 09:54:54 2016 -0700

----------------------------------------------------------------------
 .../BlockUnderConstructionFeature.java          | 45 ++++++++++++--------
 .../blockmanagement/DecommissionManager.java    |  8 +++-
 .../hdfs/server/namenode/FSDirWriteFileOp.java  | 18 +++++++-
 .../hdfs/server/namenode/NameNodeRpcServer.java |  4 ++
 4 files changed, 55 insertions(+), 20 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/5644137a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockUnderConstructionFeature.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockUnderConstructionFeature.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockUnderConstructionFeature.java
index b46b470..1a93033 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockUnderConstructionFeature.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockUnderConstructionFeature.java
@@ -33,11 +33,13 @@ import static org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCSt
  */
 public class BlockUnderConstructionFeature {
   private BlockUCState blockUCState;
+  private static final ReplicaUnderConstruction[] NO_REPLICAS =
+      new ReplicaUnderConstruction[0];
 
   /**
    * Block replicas as assigned when the block was allocated.
    */
-  private ReplicaUnderConstruction[] replicas;
+  private ReplicaUnderConstruction[] replicas = NO_REPLICAS;
 
   /**
    * Index of the primary data node doing the recovery. Useful for log
@@ -120,7 +122,7 @@ public class BlockUnderConstructionFeature {
   }
 
   public int getNumExpectedLocations() {
-    return replicas == null ? 0 : replicas.length;
+    return replicas.length;
   }
 
   /**
@@ -130,7 +132,7 @@ public class BlockUnderConstructionFeature {
    */
   void updateStorageScheduledSize(BlockInfoStriped storedBlock) {
     assert storedBlock.getUnderConstructionFeature() == this;
-    if (replicas == null) {
+    if (replicas.length == 0) {
       return;
     }
     final int dataBlockNum = storedBlock.getDataBlockNum();
@@ -182,12 +184,10 @@ public class BlockUnderConstructionFeature {
 
   List<ReplicaUnderConstruction> getStaleReplicas(long genStamp) {
     List<ReplicaUnderConstruction> staleReplicas = new ArrayList<>();
-    if (replicas != null) {
-      // Remove replicas with wrong gen stamp. The replica list is unchanged.
-      for (ReplicaUnderConstruction r : replicas) {
-        if (genStamp != r.getGenerationStamp()) {
-          staleReplicas.add(r);
-        }
+    // Remove replicas with wrong gen stamp. The replica list is unchanged.
+    for (ReplicaUnderConstruction r : replicas) {
+      if (genStamp != r.getGenerationStamp()) {
+        staleReplicas.add(r);
       }
     }
     return staleReplicas;
@@ -201,7 +201,7 @@ public class BlockUnderConstructionFeature {
   public void initializeBlockRecovery(BlockInfo blockInfo, long recoveryId) {
     setBlockUCState(BlockUCState.UNDER_RECOVERY);
     blockRecoveryId = recoveryId;
-    if (replicas == null || replicas.length == 0) {
+    if (replicas.length == 0) {
       NameNode.blockStateChangeLog.warn("BLOCK*" +
           " BlockUnderConstructionFeature.initializeBlockRecovery:" +
           " No blocks found, lease removed.");
@@ -252,7 +252,7 @@ public class BlockUnderConstructionFeature {
   /** Add the reported replica if it is not already in the replica list. */
   void addReplicaIfNotPresent(DatanodeStorageInfo storage,
       Block reportedBlock, ReplicaState rState) {
-    if (replicas == null) {
+    if (replicas.length == 0) {
       replicas = new ReplicaUnderConstruction[1];
       replicas[0] = new ReplicaUnderConstruction(reportedBlock, storage,
           rState);
@@ -295,15 +295,24 @@ public class BlockUnderConstructionFeature {
       .append(", truncateBlock=").append(truncateBlock)
       .append(", primaryNodeIndex=").append(primaryNodeIndex)
       .append(", replicas=[");
-    if (replicas != null) {
-      int i = 0;
-      for (ReplicaUnderConstruction r : replicas) {
-        r.appendStringTo(sb);
-        if (++i < replicas.length) {
-          sb.append(", ");
-        }
+    int i = 0;
+    for (ReplicaUnderConstruction r : replicas) {
+      r.appendStringTo(sb);
+      if (++i < replicas.length) {
+        sb.append(", ");
       }
     }
     sb.append("]}");
   }
+  
+  public void appendUCPartsConcise(StringBuilder sb) {
+    sb.append("replicas=");
+    int i = 0;
+    for (ReplicaUnderConstruction r : replicas) {
+      sb.append(r.getExpectedStorageLocation().getDatanodeDescriptor());
+      if (++i < replicas.length) {
+        sb.append(", ");
+      }
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5644137a/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 2a5d63c..480670a 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
@@ -37,6 +37,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.server.namenode.INodeId;
+import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.hdfs.server.namenode.Namesystem;
 import org.apache.hadoop.hdfs.util.CyclicIteration;
 import org.apache.hadoop.util.ChunkedArrayList;
@@ -280,6 +281,10 @@ public class DecommissionManager {
       BlockCollection bc,
       DatanodeDescriptor srcNode, NumberReplicas num,
       Iterable<DatanodeStorageInfo> storages) {
+    if (!NameNode.blockStateChangeLog.isInfoEnabled()) {
+      return;
+    }
+
     int curReplicas = num.liveReplicas();
     int curExpectedReplicas = blockManager.getExpectedReplicaNum(block);
     StringBuilder nodeList = new StringBuilder();
@@ -288,7 +293,8 @@ public class DecommissionManager {
       nodeList.append(node);
       nodeList.append(" ");
     }
-    LOG.info("Block: " + block + ", Expected Replicas: "
+    NameNode.blockStateChangeLog.info(
+        "Block: " + block + ", Expected Replicas: "
         + curExpectedReplicas + ", live replicas: " + curReplicas
         + ", corrupt replicas: " + num.corruptReplicas()
         + ", decommissioned replicas: " + num.decommissioned()

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5644137a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirWriteFileOp.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirWriteFileOp.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirWriteFileOp.java
index cc08528..41fd869 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirWriteFileOp.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirWriteFileOp.java
@@ -848,10 +848,26 @@ class FSDirWriteFileOp {
     assert fsn.hasWriteLock();
     BlockInfo b = addBlock(fsn.dir, src, inodesInPath, newBlock, targets,
         isStriped);
-    NameNode.stateChangeLog.info("BLOCK* allocate " + b + " for " + src);
+    logAllocatedBlock(src, b);
     DatanodeStorageInfo.incrementBlocksScheduled(targets);
   }
 
+  private static void logAllocatedBlock(String src, BlockInfo b) {
+    if (!NameNode.stateChangeLog.isInfoEnabled()) {
+      return;
+    }
+    StringBuilder sb = new StringBuilder(150);
+    sb.append("BLOCK* allocate ");
+    b.appendStringTo(sb);
+    sb.append(", ");
+    BlockUnderConstructionFeature uc = b.getUnderConstructionFeature();
+    if (uc != null) {
+      uc.appendUCPartsConcise(sb);
+    }
+    sb.append(" for " + src);
+    NameNode.stateChangeLog.info(sb.toString());
+  }
+
   private static void setNewINodeStoragePolicy(BlockManager bm, INodeFile
       inode, INodesInPath iip, boolean isLazyPersist)
       throws IOException {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5644137a/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 6dff1bc..eb47580 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
@@ -163,6 +163,7 @@ import org.apache.hadoop.ipc.RetryCache;
 import org.apache.hadoop.ipc.RetryCache.CacheEntry;
 import org.apache.hadoop.ipc.RetryCache.CacheEntryWithPayload;
 import org.apache.hadoop.ipc.Server;
+import org.apache.hadoop.ipc.StandbyException;
 import org.apache.hadoop.ipc.WritableRpcEngine;
 import org.apache.hadoop.ipc.RefreshRegistry;
 import org.apache.hadoop.ipc.RefreshResponse;
@@ -494,6 +495,9 @@ class NameNodeRpcServer implements NamenodeProtocols {
         FSLimitException.PathComponentTooLongException.class,
         FSLimitException.MaxDirectoryItemsExceededException.class,
         UnresolvedPathException.class);
+
+    clientRpcServer.addSuppressedLoggingExceptions(StandbyException.class);
+
     clientRpcServer.setTracer(nn.tracer);
     if (serviceRpcServer != null) {
       serviceRpcServer.setTracer(nn.tracer);


[42/46] hadoop git commit: HDFS-9949. Add a test case to ensure that the DataNode does not regenerate its UUID when a storage directory is cleared (Harsh J via cmccabe)

Posted by ae...@apache.org.
HDFS-9949. Add a test case to ensure that the DataNode does not regenerate its UUID when a storage directory is cleared (Harsh J via cmccabe)


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

Branch: refs/heads/HDFS-7240
Commit: dc951e606f40bb779632a8a3e3a46aeccc4a446a
Parents: ca8106d
Author: Colin Patrick Mccabe <cm...@cloudera.com>
Authored: Thu Mar 17 10:37:42 2016 -0700
Committer: Colin Patrick Mccabe <cm...@cloudera.com>
Committed: Thu Mar 17 10:37:42 2016 -0700

----------------------------------------------------------------------
 .../hdfs/server/datanode/TestDataNodeUUID.java  | 52 ++++++++++++++++++++
 1 file changed, 52 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/dc951e60/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeUUID.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeUUID.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeUUID.java
index 34e53a3..ebf7c35 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeUUID.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeUUID.java
@@ -19,17 +19,21 @@
 package org.apache.hadoop.hdfs.server.datanode;
 
 
+import org.apache.commons.io.FileUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.junit.Test;
 
+import java.io.File;
 import java.net.InetSocketAddress;
 import java.util.ArrayList;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNotEquals;
+import static org.junit.Assert.assertTrue;
 
 public class TestDataNodeUUID {
 
@@ -62,4 +66,52 @@ public class TestDataNodeUUID {
     // Make sure that we have a valid DataNodeUUID at that point of time.
     assertNotEquals(dn.getDatanodeUuid(), nullString);
   }
+
+  @Test(timeout = 10000)
+  public void testUUIDRegeneration() throws Exception {
+    File baseDir = new File(System.getProperty("test.build.data"));
+    File disk1 = new File(baseDir, "disk1");
+    File disk2 = new File(baseDir, "disk2");
+
+    // Ensure the configured disks do not pre-exist
+    FileUtils.deleteDirectory(disk1);
+    FileUtils.deleteDirectory(disk2);
+
+    MiniDFSCluster cluster = null;
+    HdfsConfiguration conf = new HdfsConfiguration();
+    conf.setStrings(DFSConfigKeys.DFS_DATANODE_DATA_DIR_KEY,
+            disk1.toURI().toString(),
+            disk2.toURI().toString());
+    try {
+      cluster = new MiniDFSCluster.Builder(conf)
+              .numDataNodes(1)
+              .manageDataDfsDirs(false)
+              .build();
+      cluster.waitActive();
+
+      // Grab the new-cluster UUID as the original one to test against
+      String originalUUID = cluster.getDataNodes().get(0).getDatanodeUuid();
+      // Stop and simulate a DN wipe or unmount-but-root-path condition
+      // on the second disk
+      MiniDFSCluster.DataNodeProperties dn = cluster.stopDataNode(0);
+      FileUtils.deleteDirectory(disk2);
+      assertTrue("Failed to recreate the data directory: " + disk2,
+              disk2.mkdirs());
+
+      // Restart and check if the UUID changed
+      assertTrue("DataNode failed to start up: " + dn,
+              cluster.restartDataNode(dn));
+      // We need to wait until the DN has completed registration
+      while (!cluster.getDataNodes().get(0).isDatanodeFullyStarted()) {
+        Thread.sleep(50);
+      }
+      assertEquals(
+              "DN generated a new UUID despite disk1 having it intact",
+              originalUUID, cluster.getDataNodes().get(0).getDatanodeUuid());
+    } finally {
+      if (cluster != null) {
+        cluster.shutdown();
+      }
+    }
+  }
 }


[30/46] hadoop git commit: CapacityScheduler: Improve preemption to only kill containers that would satisfy the incoming request. (Wangda Tan)

Posted by ae...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/7e8c9beb/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/preemption/PreemptableQueue.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/preemption/PreemptableQueue.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/preemption/PreemptableQueue.java
new file mode 100644
index 0000000..19148d7
--- /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/preemption/PreemptableQueue.java
@@ -0,0 +1,102 @@
+/**
+ * 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
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
+ * 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.preemption;
+
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
+import org.apache.hadoop.yarn.util.resource.Resources;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.ConcurrentSkipListMap;
+
+public class PreemptableQueue {
+  // Partition -> killable resources and containers
+  private Map<String, Resource> totalKillableResources = new HashMap<>();
+  private Map<String, Map<ContainerId, RMContainer>> killableContainers =
+      new HashMap<>();
+  private PreemptableQueue parent;
+
+  public PreemptableQueue(PreemptableQueue parent) {
+    this.parent = parent;
+  }
+
+  public PreemptableQueue(Map<String, Resource> totalKillableResources,
+      Map<String, Map<ContainerId, RMContainer>> killableContainers) {
+    this.totalKillableResources = totalKillableResources;
+    this.killableContainers = killableContainers;
+  }
+
+  void addKillableContainer(KillableContainer container) {
+    String partition = container.getNodePartition();
+    if (!totalKillableResources.containsKey(partition)) {
+      totalKillableResources.put(partition, Resources.createResource(0));
+      killableContainers.put(partition,
+          new ConcurrentSkipListMap<ContainerId, RMContainer>());
+    }
+
+    RMContainer c = container.getRMContainer();
+    Resources.addTo(totalKillableResources.get(partition),
+        c.getAllocatedResource());
+    killableContainers.get(partition).put(c.getContainerId(), c);
+
+    if (null != parent) {
+      parent.addKillableContainer(container);
+    }
+  }
+
+  void removeKillableContainer(KillableContainer container) {
+    String partition = container.getNodePartition();
+    Map<ContainerId, RMContainer> partitionKillableContainers =
+        killableContainers.get(partition);
+    if (partitionKillableContainers != null) {
+      RMContainer rmContainer = partitionKillableContainers.remove(
+          container.getRMContainer().getContainerId());
+      if (null != rmContainer) {
+        Resources.subtractFrom(totalKillableResources.get(partition),
+            rmContainer.getAllocatedResource());
+      }
+    }
+
+    if (null != parent) {
+      parent.removeKillableContainer(container);
+    }
+  }
+
+  public Resource getKillableResource(String partition) {
+    Resource res = totalKillableResources.get(partition);
+    return res == null ? Resources.none() : res;
+  }
+
+  @SuppressWarnings("unchecked")
+  public Map<ContainerId, RMContainer> getKillableContainers(String partition) {
+    Map<ContainerId, RMContainer> map = killableContainers.get(partition);
+    return map == null ? Collections.EMPTY_MAP : map;
+  }
+
+  public Map<String, Map<ContainerId, RMContainer>> getKillableContainers() {
+    return killableContainers;
+  }
+
+  Map<String, Resource> getTotalKillableResources() {
+    return totalKillableResources;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7e8c9beb/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/preemption/PreemptionManager.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/preemption/PreemptionManager.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/preemption/PreemptionManager.java
new file mode 100644
index 0000000..a9f02a5
--- /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/preemption/PreemptionManager.java
@@ -0,0 +1,165 @@
+/**
+ * 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
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
+ * 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.preemption;
+
+import com.google.common.annotations.VisibleForTesting;
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CSQueue;
+import org.apache.hadoop.yarn.util.resource.Resources;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+
+public class PreemptionManager {
+  private ReentrantReadWriteLock.ReadLock readLock;
+  private ReentrantReadWriteLock.WriteLock writeLock;
+  private Map<String, PreemptableQueue> entities = new HashMap<>();
+
+  public PreemptionManager() {
+    ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
+    readLock = lock.readLock();
+    writeLock = lock.writeLock();
+  }
+
+  public void refreshQueues(CSQueue parent, CSQueue current) {
+    try {
+      writeLock.lock();
+      PreemptableQueue parentEntity = null;
+      if (parent != null) {
+        parentEntity = entities.get(parent.getQueueName());
+      }
+
+      if (!entities.containsKey(current.getQueueName())) {
+        entities.put(current.getQueueName(),
+            new PreemptableQueue(parentEntity));
+      }
+
+      if (current.getChildQueues() != null) {
+        for (CSQueue child : current.getChildQueues()) {
+          refreshQueues(current, child);
+        }
+      }
+    }
+    finally {
+      writeLock.unlock();
+    }
+  }
+
+  public void addKillableContainer(KillableContainer container) {
+    try {
+      writeLock.lock();
+      PreemptableQueue entity = entities.get(container.getLeafQueueName());
+      if (null != entity) {
+        entity.addKillableContainer(container);
+      }
+    }
+    finally {
+      writeLock.unlock();
+    }
+  }
+
+  public void removeKillableContainer(KillableContainer container) {
+    try {
+      writeLock.lock();
+      PreemptableQueue entity = entities.get(container.getLeafQueueName());
+      if (null != entity) {
+        entity.removeKillableContainer(container);
+      }
+    }
+    finally {
+      writeLock.unlock();
+    }
+  }
+
+  public void moveKillableContainer(KillableContainer oldContainer,
+      KillableContainer newContainer) {
+    // TODO, will be called when partition of the node changed OR
+    // container moved to different queue
+  }
+
+  public void updateKillableContainerResource(KillableContainer container,
+      Resource oldResource, Resource newResource) {
+    // TODO, will be called when container's resource changed
+  }
+
+  @VisibleForTesting
+  public Map<ContainerId, RMContainer> getKillableContainersMap(
+      String queueName, String partition) {
+    try {
+      readLock.lock();
+      PreemptableQueue entity = entities.get(queueName);
+      if (entity != null) {
+        Map<ContainerId, RMContainer> containers =
+            entity.getKillableContainers().get(partition);
+        if (containers != null) {
+          return containers;
+        }
+      }
+      return Collections.emptyMap();
+    }
+    finally {
+      readLock.unlock();
+    }
+  }
+
+  public Iterator<RMContainer> getKillableContainers(String queueName,
+      String partition) {
+    return getKillableContainersMap(queueName, partition).values().iterator();
+  }
+
+  public Resource getKillableResource(String queueName, String partition) {
+    try {
+      readLock.lock();
+      PreemptableQueue entity = entities.get(queueName);
+      if (entity != null) {
+        Resource res = entity.getTotalKillableResources().get(partition);
+        if (res == null || res.equals(Resources.none())) {
+          return Resources.none();
+        }
+        return Resources.clone(res);
+      }
+      return Resources.none();
+    }
+    finally {
+      readLock.unlock();
+    }
+  }
+
+  public Map<String, PreemptableQueue> getShallowCopyOfPreemptableEntities() {
+    try {
+      readLock.lock();
+      Map<String, PreemptableQueue> map = new HashMap<>();
+      for (Map.Entry<String, PreemptableQueue> entry : entities.entrySet()) {
+        String key = entry.getKey();
+        PreemptableQueue entity = entry.getValue();
+        map.put(key, new PreemptableQueue(
+            new HashMap<>(entity.getTotalKillableResources()),
+            new HashMap<>(entity.getKillableContainers())));
+      }
+      return map;
+    } finally {
+      readLock.unlock();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7e8c9beb/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/common/AssignmentInformation.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/AssignmentInformation.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/AssignmentInformation.java
index 5158255..aad3bc7 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/AssignmentInformation.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/AssignmentInformation.java
@@ -120,9 +120,9 @@ public class AssignmentInformation {
   }
 
   private ContainerId getFirstContainerIdFromOperation(Operation op) {
-    if (null != operationDetails.get(Operation.ALLOCATION)) {
+    if (null != operationDetails.get(op)) {
       List<AssignmentDetails> assignDetails =
-          operationDetails.get(Operation.ALLOCATION);
+          operationDetails.get(op);
       if (!assignDetails.isEmpty()) {
         return assignDetails.get(0).containerId;
       }
@@ -131,7 +131,7 @@ public class AssignmentInformation {
   }
 
   public ContainerId getFirstAllocatedOrReservedContainerId() {
-    ContainerId containerId = null;
+    ContainerId containerId;
     containerId = getFirstContainerIdFromOperation(Operation.ALLOCATION);
     if (null != containerId) {
       return containerId;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7e8c9beb/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 4d563cd..f474aad 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
@@ -59,6 +59,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CSAMCont
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CSAssignment;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityHeadroomProvider;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerContext;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.LeafQueue;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueueCapacities;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.SchedulingMode;
@@ -94,6 +95,7 @@ public class FiCaSchedulerApp extends SchedulerApplicationAttempt {
    * to hold the message if its app doesn't not get container from a node
    */
   private String appSkipNodeDiagnostics;
+  private CapacitySchedulerContext capacitySchedulerContext;
 
   public FiCaSchedulerApp(ApplicationAttemptId applicationAttemptId, 
       String user, Queue queue, ActiveUsersManager activeUsersManager,
@@ -138,28 +140,30 @@ public class FiCaSchedulerApp extends SchedulerApplicationAttempt {
     }
     
     containerAllocator = new ContainerAllocator(this, rc, rmContext);
+
+    if (scheduler instanceof CapacityScheduler) {
+      capacitySchedulerContext = (CapacitySchedulerContext) scheduler;
+    }
   }
 
-  synchronized public boolean containerCompleted(RMContainer rmContainer,
+  public synchronized boolean containerCompleted(RMContainer rmContainer,
       ContainerStatus containerStatus, RMContainerEventType event,
       String partition) {
+    ContainerId containerId = rmContainer.getContainerId();
 
     // Remove from the list of containers
-    if (null == liveContainers.remove(rmContainer.getContainerId())) {
+    if (null == liveContainers.remove(containerId)) {
       return false;
     }
-    
+
     // Remove from the list of newly allocated containers if found
     newlyAllocatedContainers.remove(rmContainer);
 
-    Container container = rmContainer.getContainer();
-    ContainerId containerId = container.getId();
-
     // Inform the container
     rmContainer.handle(
         new RMContainerFinishedEvent(containerId, containerStatus, event));
 
-    containersToPreempt.remove(rmContainer.getContainerId());
+    containersToPreempt.remove(containerId);
 
     RMAuditLogger.logSuccess(getUser(),
         AuditConstants.RELEASE_CONTAINER, "SchedulerApp",
@@ -176,7 +180,7 @@ public class FiCaSchedulerApp extends SchedulerApplicationAttempt {
     return true;
   }
 
-  synchronized public RMContainer allocate(NodeType type, FiCaSchedulerNode node,
+  public synchronized RMContainer allocate(NodeType type, FiCaSchedulerNode node,
       Priority priority, ResourceRequest request, 
       Container container) {
 
@@ -200,7 +204,9 @@ public class FiCaSchedulerApp extends SchedulerApplicationAttempt {
 
     // Add it to allContainers list.
     newlyAllocatedContainers.add(rmContainer);
-    liveContainers.put(container.getId(), rmContainer);    
+
+    ContainerId containerId = container.getId();
+    liveContainers.put(containerId, rmContainer);
 
     // Update consumption and track allocations
     List<ResourceRequest> resourceRequestList = appSchedulingInfo.allocate(
@@ -213,17 +219,17 @@ public class FiCaSchedulerApp extends SchedulerApplicationAttempt {
 
     // Inform the container
     rmContainer.handle(
-        new RMContainerEvent(container.getId(), RMContainerEventType.START));
+        new RMContainerEvent(containerId, RMContainerEventType.START));
 
     if (LOG.isDebugEnabled()) {
       LOG.debug("allocate: applicationAttemptId=" 
-          + container.getId().getApplicationAttemptId() 
-          + " container=" + container.getId() + " host="
+          + containerId.getApplicationAttemptId()
+          + " container=" + containerId + " host="
           + container.getNodeId().getHost() + " type=" + type);
     }
     RMAuditLogger.logSuccess(getUser(),
         AuditConstants.ALLOC_CONTAINER, "SchedulerApp",
-        getApplicationId(), container.getId());
+        getApplicationId(), containerId);
     
     return rmContainer;
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7e8c9beb/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/common/fica/FiCaSchedulerNode.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/FiCaSchedulerNode.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/FiCaSchedulerNode.java
index fe6db47..1d0e78a 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/FiCaSchedulerNode.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/FiCaSchedulerNode.java
@@ -18,22 +18,29 @@
 
 package org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica;
 
-
-import java.util.Set;
-
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
+import org.apache.hadoop.yarn.api.records.Container;
+import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.Priority;
+import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.nodelabels.CommonNodeLabelsManager;
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplicationAttempt;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNode;
+import org.apache.hadoop.yarn.util.resource.Resources;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Set;
 
 public class FiCaSchedulerNode extends SchedulerNode {
 
   private static final Log LOG = LogFactory.getLog(FiCaSchedulerNode.class);
+  private Map<ContainerId, RMContainer> killableContainers = new HashMap<>();
+  private Resource totalKillableResources = Resource.newInstance(0, 0);
   
   public FiCaSchedulerNode(RMNode node, boolean usePortForNodeName,
       Set<String> nodeLabels) {
@@ -92,7 +99,6 @@ public class FiCaSchedulerNode extends SchedulerNode {
   @Override
   public synchronized void unreserveResource(
       SchedulerApplicationAttempt application) {
-
     // adding NP checks as this can now be called for preemption
     if (getReservedContainer() != null
         && getReservedContainer().getContainer() != null
@@ -115,4 +121,55 @@ public class FiCaSchedulerNode extends SchedulerNode {
     }
     setReservedContainer(null);
   }
+
+  // According to decisions from preemption policy, mark the container to killable
+  public synchronized void markContainerToKillable(ContainerId containerId) {
+    RMContainer c = launchedContainers.get(containerId);
+    if (c != null && !killableContainers.containsKey(containerId)) {
+      killableContainers.put(containerId, c);
+      Resources.addTo(totalKillableResources, c.getAllocatedResource());
+    }
+  }
+
+  // According to decisions from preemption policy, mark the container to
+  // non-killable
+  public synchronized void markContainerToNonKillable(ContainerId containerId) {
+    RMContainer c = launchedContainers.get(containerId);
+    if (c != null && killableContainers.containsKey(containerId)) {
+      killableContainers.remove(containerId);
+      Resources.subtractFrom(totalKillableResources, c.getAllocatedResource());
+    }
+  }
+
+  @Override
+  protected synchronized void updateResource(
+      Container container) {
+    super.updateResource(container);
+    if (killableContainers.containsKey(container.getId())) {
+      Resources.subtractFrom(totalKillableResources, container.getResource());
+      killableContainers.remove(container.getId());
+    }
+  }
+
+  @Override
+  protected synchronized void changeContainerResource(ContainerId containerId,
+      Resource deltaResource, boolean increase) {
+    super.changeContainerResource(containerId, deltaResource, increase);
+
+    if (killableContainers.containsKey(containerId)) {
+      if (increase) {
+        Resources.addTo(totalKillableResources, deltaResource);
+      } else {
+        Resources.subtractFrom(totalKillableResources, deltaResource);
+      }
+    }
+  }
+
+  public synchronized Resource getTotalKillableResources() {
+    return totalKillableResources;
+  }
+
+  public synchronized Map<ContainerId, RMContainer> getKillableContainers() {
+    return killableContainers;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7e8c9beb/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/event/SchedulerEventType.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/event/SchedulerEventType.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/event/SchedulerEventType.java
index 9cf09e9..35b7c14 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/event/SchedulerEventType.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/event/SchedulerEventType.java
@@ -38,10 +38,15 @@ public enum SchedulerEventType {
   // Source: ContainerAllocationExpirer
   CONTAINER_EXPIRED,
 
-  // Source: SchedulingEditPolicy
+  /* Source: SchedulingEditPolicy */
   KILL_RESERVED_CONTAINER,
-  MARK_CONTAINER_FOR_PREEMPTION, // Mark a container for preemption
-                                 // in the near future
-  KILL_PREEMPTED_CONTAINER // Kill a container previously marked for
-                           // preemption
+
+  // Mark a container for preemption
+  MARK_CONTAINER_FOR_PREEMPTION,
+
+  // Mark a for-preemption container killable
+  MARK_CONTAINER_FOR_KILLABLE,
+
+  // Cancel a killable container
+  MARK_CONTAINER_FOR_NONKILLABLE
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7e8c9beb/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMDispatcher.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/TestRMDispatcher.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMDispatcher.java
index d9306dd..c944752 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMDispatcher.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMDispatcher.java
@@ -59,7 +59,7 @@ public class TestRMDispatcher {
       rmDispatcher.getEventHandler().handle(event1);
       ContainerPreemptEvent event2 =
           new ContainerPreemptEvent(appAttemptId, container,
-            SchedulerEventType.KILL_PREEMPTED_CONTAINER);
+            SchedulerEventType.MARK_CONTAINER_FOR_KILLABLE);
       rmDispatcher.getEventHandler().handle(event2);
       ContainerPreemptEvent event3 =
           new ContainerPreemptEvent(appAttemptId, container,
@@ -70,7 +70,7 @@ public class TestRMDispatcher {
       verify(sched, times(3)).handle(any(SchedulerEvent.class));
       verify(sched).killReservedContainer(container);
       verify(sched).markContainerForPreemption(appAttemptId, container);
-      verify(sched).killPreemptedContainer(container);
+      verify(sched).markContainerForKillable(container);
     } catch (InterruptedException e) {
       Assert.fail();
     } finally {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7e8c9beb/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 028afb1..3057615 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
@@ -2352,7 +2352,7 @@ public class TestRMRestart extends ParameterizedSchedulerTestBase {
       FiCaSchedulerApp schedulerAppAttempt = cs.getSchedulerApplications()
           .get(app0.getApplicationId()).getCurrentAppAttempt();
       // kill app0-attempt
-      cs.killPreemptedContainer(schedulerAppAttempt.getRMContainer(
+      cs.markContainerForKillable(schedulerAppAttempt.getRMContainer(
           app0.getCurrentAppAttempt().getMasterContainer().getId()));
       am0.waitForState(RMAppAttemptState.FAILED);
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7e8c9beb/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/applicationsmanager/TestAMRestart.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/TestAMRestart.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/applicationsmanager/TestAMRestart.java
index 5035afe..16f3f60 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/applicationsmanager/TestAMRestart.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/applicationsmanager/TestAMRestart.java
@@ -63,7 +63,6 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEv
 import org.apache.hadoop.yarn.server.utils.BuilderUtils;
 import org.apache.hadoop.yarn.util.ControlledClock;
 import org.apache.hadoop.yarn.util.Records;
-import org.apache.hadoop.yarn.util.SystemClock;
 import org.apache.hadoop.yarn.util.resource.Resources;
 import org.junit.Assert;
 import org.junit.Test;
@@ -566,7 +565,7 @@ public class TestAMRestart {
     ContainerId amContainer =
         ContainerId.newContainerId(am1.getApplicationAttemptId(), 1);
     // Preempt the first attempt;
-    scheduler.killPreemptedContainer(scheduler.getRMContainer(amContainer));
+    scheduler.markContainerForKillable(scheduler.getRMContainer(amContainer));
 
     am1.waitForState(RMAppAttemptState.FAILED);
     Assert.assertTrue(! attempt1.shouldCountTowardsMaxAttemptRetry());
@@ -582,7 +581,7 @@ public class TestAMRestart {
     // Preempt the second attempt.
     ContainerId amContainer2 =
         ContainerId.newContainerId(am2.getApplicationAttemptId(), 1);
-    scheduler.killPreemptedContainer(scheduler.getRMContainer(amContainer2));
+    scheduler.markContainerForKillable(scheduler.getRMContainer(amContainer2));
 
     am2.waitForState(RMAppAttemptState.FAILED);
     Assert.assertTrue(! attempt2.shouldCountTowardsMaxAttemptRetry());
@@ -677,7 +676,7 @@ public class TestAMRestart {
         ContainerId.newContainerId(am1.getApplicationAttemptId(), 1);
 
     // Forcibly preempt the am container;
-    scheduler.killPreemptedContainer(scheduler.getRMContainer(amContainer));
+    scheduler.markContainerForKillable(scheduler.getRMContainer(amContainer));
 
     am1.waitForState(RMAppAttemptState.FAILED);
     Assert.assertTrue(! attempt1.shouldCountTowardsMaxAttemptRetry());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7e8c9beb/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/TestProportionalCapacityPreemptionPolicy.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/monitor/capacity/TestProportionalCapacityPreemptionPolicy.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/TestProportionalCapacityPreemptionPolicy.java
index 13f267d..e9129de 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/TestProportionalCapacityPreemptionPolicy.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/TestProportionalCapacityPreemptionPolicy.java
@@ -23,7 +23,7 @@ import static org.apache.hadoop.yarn.server.resourcemanager.monitor.capacity.Pro
 import static org.apache.hadoop.yarn.server.resourcemanager.monitor.capacity.ProportionalCapacityPreemptionPolicy.OBSERVE_ONLY;
 import static org.apache.hadoop.yarn.server.resourcemanager.monitor.capacity.ProportionalCapacityPreemptionPolicy.TOTAL_PREEMPTION_PER_ROUND;
 import static org.apache.hadoop.yarn.server.resourcemanager.monitor.capacity.ProportionalCapacityPreemptionPolicy.WAIT_TIME_BEFORE_KILL;
-import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEventType.KILL_PREEMPTED_CONTAINER;
+import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEventType.MARK_CONTAINER_FOR_KILLABLE;
 import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEventType.MARK_CONTAINER_FOR_PREEMPTION;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNotNull;
@@ -75,6 +75,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.Capacity
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.LeafQueue;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.ParentQueue;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueueCapacities;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.preemption.PreemptionManager;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerApp;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.ContainerPreemptEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEvent;
@@ -167,6 +168,7 @@ public class TestProportionalCapacityPreemptionPolicy {
     when(mCS.getConfiguration()).thenReturn(schedConf);
     rmContext = mock(RMContext.class);
     when(mCS.getRMContext()).thenReturn(rmContext);
+    when(mCS.getPreemptionManager()).thenReturn(new PreemptionManager());
     when(rmContext.getNodeLabelManager()).thenReturn(lm);
     mDisp = mock(EventHandler.class);
     Dispatcher disp = mock(Dispatcher.class);
@@ -289,7 +291,7 @@ public class TestProportionalCapacityPreemptionPolicy {
     List<ContainerPreemptEvent> events = evtCaptor.getAllValues();
     for (ContainerPreemptEvent e : events.subList(20, 20)) {
       assertEquals(appC, e.getAppId());
-      assertEquals(KILL_PREEMPTED_CONTAINER, e.getType());
+      assertEquals(MARK_CONTAINER_FOR_KILLABLE, e.getType());
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7e8c9beb/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/TestProportionalCapacityPreemptionPolicyForNodePartitions.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/monitor/capacity/TestProportionalCapacityPreemptionPolicyForNodePartitions.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/TestProportionalCapacityPreemptionPolicyForNodePartitions.java
index 512f37c..21ea495 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/TestProportionalCapacityPreemptionPolicyForNodePartitions.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/TestProportionalCapacityPreemptionPolicyForNodePartitions.java
@@ -67,6 +67,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.Capacity
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.LeafQueue;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.ParentQueue;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueueCapacities;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.preemption.PreemptionManager;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerApp;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.policy.OrderingPolicy;
@@ -123,6 +124,7 @@ public class TestProportionalCapacityPreemptionPolicyForNodePartitions {
     mClock = mock(Clock.class);
     cs = mock(CapacityScheduler.class);
     when(cs.getResourceCalculator()).thenReturn(rc);
+    when(cs.getPreemptionManager()).thenReturn(new PreemptionManager());
 
     nlm = mock(RMNodeLabelsManager.class);
     mDisp = mock(EventHandler.class);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7e8c9beb/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 0b32676..171196f 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
@@ -52,6 +52,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttempt;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ActiveUsersManager;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceLimits;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.preemption.PreemptionManager;
 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.server.resourcemanager.security.RMContainerTokenSecretManager;
@@ -264,6 +265,7 @@ public class TestApplicationLimits {
         thenReturn(CapacityScheduler.nonPartitionedQueueComparator);
     when(csContext.getResourceCalculator()).thenReturn(resourceCalculator);
     when(csContext.getRMContext()).thenReturn(rmContext);
+    when(csContext.getPreemptionManager()).thenReturn(new PreemptionManager());
     
     // Say cluster has 100 nodes of 16G each
     Resource clusterResource = 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7e8c9beb/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestApplicationPriority.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/TestApplicationPriority.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/TestApplicationPriority.java
index 1569a12..d8161f8 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/TestApplicationPriority.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/TestApplicationPriority.java
@@ -205,7 +205,7 @@ public class TestApplicationPriority {
       if (++counter > 2) {
         break;
       }
-      cs.killPreemptedContainer(schedulerAppAttempt.getRMContainer(c.getId()));
+      cs.markContainerForKillable(schedulerAppAttempt.getRMContainer(c.getId()));
     }
 
     // check node report, 12 GB used and 4 GB available
@@ -512,7 +512,7 @@ public class TestApplicationPriority {
       if (++counter > 2) {
         break;
       }
-      cs.killPreemptedContainer(schedulerAppAttemptApp1.getRMContainer(c.getId()));
+      cs.markContainerForKillable(schedulerAppAttemptApp1.getRMContainer(c.getId()));
       iterator.remove();
     }
 
@@ -542,7 +542,7 @@ public class TestApplicationPriority {
       if (++counter > 1) {
         break;
       }
-      cs.killPreemptedContainer(schedulerAppAttemptApp1.getRMContainer(c.getId()));
+      cs.markContainerForKillable(schedulerAppAttemptApp1.getRMContainer(c.getId()));
       iterator.remove();
     }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7e8c9beb/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacityScheduler.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/TestCapacityScheduler.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/TestCapacityScheduler.java
index b6c005b..16ba607 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/TestCapacityScheduler.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/TestCapacityScheduler.java
@@ -1188,7 +1188,7 @@ public class TestCapacityScheduler {
 
     // kill the 3 containers
     for (Container c : allocatedContainers) {
-      cs.killPreemptedContainer(schedulerAppAttempt.getRMContainer(c.getId()));
+      cs.markContainerForKillable(schedulerAppAttempt.getRMContainer(c.getId()));
     }
 
     // check values
@@ -1197,7 +1197,7 @@ public class TestCapacityScheduler {
         Resource.newInstance(CONTAINER_MEMORY * 3, 3), false, 3);
 
     // kill app0-attempt0 AM container
-    cs.killPreemptedContainer(schedulerAppAttempt.getRMContainer(app0
+    cs.markContainerForKillable(schedulerAppAttempt.getRMContainer(app0
         .getCurrentAppAttempt().getMasterContainer().getId()));
 
     // wait for app0 failed
@@ -1220,7 +1220,7 @@ public class TestCapacityScheduler {
     allocatedContainers =
         am1.allocateAndWaitForContainers(3, CONTAINER_MEMORY, nm1);
     for (Container c : allocatedContainers) {
-      cs.killPreemptedContainer(schedulerAppAttempt.getRMContainer(c.getId()));
+      cs.markContainerForKillable(schedulerAppAttempt.getRMContainer(c.getId()));
     }
 
     // check values
@@ -1269,7 +1269,7 @@ public class TestCapacityScheduler {
     }
 
     // Call killContainer to preempt the container
-    cs.killPreemptedContainer(rmContainer);
+    cs.markContainerForKillable(rmContainer);
 
     Assert.assertEquals(3, requests.size());
     for (ResourceRequest request : requests) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7e8c9beb/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacitySchedulerPreemption.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/TestCapacitySchedulerPreemption.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/TestCapacitySchedulerPreemption.java
new file mode 100644
index 0000000..bea7797
--- /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/TestCapacitySchedulerPreemption.java
@@ -0,0 +1,677 @@
+/**
+ * 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 com.google.common.collect.Sets;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.service.Service;
+import org.apache.hadoop.yarn.api.records.ContainerId;
+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.RMActiveServices;
+import org.apache.hadoop.yarn.server.resourcemanager.monitor.SchedulingEditPolicy;
+import org.apache.hadoop.yarn.server.resourcemanager.monitor.SchedulingMonitor;
+import org.apache.hadoop.yarn.server.resourcemanager.monitor.capacity.ProportionalCapacityPreemptionPolicy;
+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.rmnode.RMNode;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.preemption.PreemptionManager;
+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.util.Clock;
+import org.apache.hadoop.yarn.util.resource.Resources;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+public class TestCapacitySchedulerPreemption {
+  private static final Log LOG = LogFactory.getLog(
+      TestCapacitySchedulerPreemption.class);
+
+  private final int GB = 1024;
+
+  private Configuration conf;
+
+  RMNodeLabelsManager mgr;
+
+  Clock clock;
+
+  @Before
+  public void setUp() throws Exception {
+    conf = new YarnConfiguration();
+    conf.setClass(YarnConfiguration.RM_SCHEDULER, CapacityScheduler.class,
+        ResourceScheduler.class);
+    conf.setBoolean(YarnConfiguration.RM_SCHEDULER_ENABLE_MONITORS, true);
+    conf.setClass(YarnConfiguration.RM_SCHEDULER_MONITOR_POLICIES,
+        ProportionalCapacityPreemptionPolicy.class, SchedulingEditPolicy.class);
+    conf = TestUtils.getConfigurationWithMultipleQueues(this.conf);
+
+    // Set preemption related configurations
+    conf.setInt(ProportionalCapacityPreemptionPolicy.WAIT_TIME_BEFORE_KILL,
+        0);
+    conf.setBoolean(CapacitySchedulerConfiguration.LAZY_PREEMPTION_ENALBED,
+        true);
+    conf.setFloat(
+        ProportionalCapacityPreemptionPolicy.TOTAL_PREEMPTION_PER_ROUND, 1.0f);
+    conf.setFloat(
+        ProportionalCapacityPreemptionPolicy.NATURAL_TERMINATION_FACTOR, 1.0f);
+    mgr = new NullRMNodeLabelsManager();
+    mgr.init(this.conf);
+    clock = mock(Clock.class);
+    when(clock.getTime()).thenReturn(0L);
+  }
+
+  private SchedulingEditPolicy getSchedulingEditPolicy(MockRM rm) {
+    RMActiveServices activeServices = rm.getRMActiveService();
+    SchedulingMonitor mon = null;
+    for (Service service : activeServices.getServices()) {
+      if (service instanceof SchedulingMonitor) {
+        mon = (SchedulingMonitor) service;
+        break;
+      }
+    }
+
+    if (mon != null) {
+      return mon.getSchedulingEditPolicy();
+    }
+    return null;
+  }
+
+  @Test (timeout = 60000)
+  public void testSimplePreemption() throws Exception {
+    /**
+     * Test case: Submit two application (app1/app2) to different queues, queue
+     * structure:
+     *
+     * <pre>
+     *             Root
+     *            /  |  \
+     *           a   b   c
+     *          10   20  70
+     * </pre>
+     *
+     * 1) Two nodes in the cluster, each of them has 4G.
+     *
+     * 2) app1 submit to queue-a first, it asked 7 * 1G containers, so there's no
+     * more resource available.
+     *
+     * 3) app2 submit to queue-c, ask for one 1G container (for AM)
+     *
+     * Now the cluster is fulfilled.
+     *
+     * 4) app2 asks for another 1G container, system will preempt one container
+     * from app1, and app2 will receive the preempted container
+     */
+    MockRM rm1 = new MockRM(conf);
+    rm1.getRMContext().setNodeLabelManager(mgr);
+    rm1.start();
+
+    MockNM nm1 = rm1.registerNode("h1:1234", 4 * GB);
+    MockNM nm2 = rm1.registerNode("h2:1234", 4 * GB);
+    CapacityScheduler cs = (CapacityScheduler) rm1.getResourceScheduler();
+    RMNode rmNode1 = rm1.getRMContext().getRMNodes().get(nm1.getNodeId());
+    RMNode rmNode2 = rm1.getRMContext().getRMNodes().get(nm2.getNodeId());
+
+    // launch an app to queue, AM container should be launched in nm1
+    RMApp app1 = rm1.submitApp(1 * GB, "app", "user", null, "a");
+    MockAM am1 = MockRM.launchAndRegisterAM(app1, rm1, nm1);
+
+    am1.allocate("*", 1 * GB, 7, new ArrayList<ContainerId>());
+
+    // Do allocation 3 times for node1/node2
+    for (int i = 0; i < 3; i++) {
+      cs.handle(new NodeUpdateSchedulerEvent(rmNode1));
+      cs.handle(new NodeUpdateSchedulerEvent(rmNode2));
+    }
+
+    // App1 should have 7 containers now, and no available resource for cluster
+    FiCaSchedulerApp schedulerApp1 = cs.getApplicationAttempt(
+        am1.getApplicationAttemptId());
+    Assert.assertEquals(7, schedulerApp1.getLiveContainers().size());
+
+    // Submit app2 to queue-c and asks for a 1G container for AM
+    RMApp app2 = rm1.submitApp(1 * GB, "app", "user", null, "c");
+    MockAM am2 = MockRM.launchAndRegisterAM(app2, rm1, nm2);
+
+    // NM1/NM2 has available resource = 0G
+    Assert.assertEquals(0 * GB, cs.getNode(nm1.getNodeId())
+        .getUnallocatedResource().getMemory());
+    Assert.assertEquals(0 * GB, cs.getNode(nm2.getNodeId())
+        .getUnallocatedResource().getMemory());
+
+    // AM asks for a 1 * GB container
+    am2.allocate(Arrays.asList(ResourceRequest
+        .newInstance(Priority.newInstance(1), ResourceRequest.ANY,
+            Resources.createResource(1 * GB), 1)), null);
+
+    // Get edit policy and do one update
+    SchedulingEditPolicy editPolicy = getSchedulingEditPolicy(rm1);
+
+    // Call edit schedule twice, and check if one container from app1 marked
+    // to be "killable"
+    editPolicy.editSchedule();
+    editPolicy.editSchedule();
+
+    PreemptionManager pm = cs.getPreemptionManager();
+    Map<ContainerId, RMContainer> killableContainers =
+        waitKillableContainersSize(pm, "a", RMNodeLabelsManager.NO_LABEL, 1);
+    Assert.assertEquals(1, killableContainers.size());
+    Assert.assertEquals(killableContainers.entrySet().iterator().next().getKey()
+        .getApplicationAttemptId(), am1.getApplicationAttemptId());
+
+    // Call CS.handle once to see if container preempted
+    cs.handle(new NodeUpdateSchedulerEvent(rmNode2));
+
+    FiCaSchedulerApp schedulerApp2 = cs.getApplicationAttempt(
+        am2.getApplicationAttemptId());
+
+    // App1 has 6 containers, and app2 has 2 containers
+    Assert.assertEquals(6, schedulerApp1.getLiveContainers().size());
+    Assert.assertEquals(2, schedulerApp2.getLiveContainers().size());
+
+    rm1.close();
+  }
+
+  @Test (timeout = 60000)
+  public void testPreemptionConsidersNodeLocalityDelay()
+      throws Exception {
+    /**
+     * Test case: same as testSimplePreemption steps 1-3.
+     *
+     * Step 4: app2 asks for 1G container with locality specified, so it needs
+     * to wait for missed-opportunity before get scheduled.
+     * Check if system waits missed-opportunity before finish killable container
+     */
+    MockRM rm1 = new MockRM(conf);
+    rm1.getRMContext().setNodeLabelManager(mgr);
+    rm1.start();
+    MockNM nm1 = rm1.registerNode("h1:1234", 4 * GB);
+    MockNM nm2 = rm1.registerNode("h2:1234", 4 * GB);
+    CapacityScheduler cs = (CapacityScheduler) rm1.getResourceScheduler();
+    RMNode rmNode1 = rm1.getRMContext().getRMNodes().get(nm1.getNodeId());
+    RMNode rmNode2 = rm1.getRMContext().getRMNodes().get(nm2.getNodeId());
+
+    // launch an app to queue, AM container should be launched in nm1
+    RMApp app1 = rm1.submitApp(1 * GB, "app", "user", null, "a");
+    MockAM am1 = MockRM.launchAndRegisterAM(app1, rm1, nm1);
+
+    am1.allocate("*", 1 * GB, 6, new ArrayList<ContainerId>());
+
+    // Do allocation 3 times for node1/node2
+    for (int i = 0; i < 3; i++) {
+      cs.handle(new NodeUpdateSchedulerEvent(rmNode1));
+      cs.handle(new NodeUpdateSchedulerEvent(rmNode2));
+    }
+
+    // App1 should have 7 containers now, and no available resource for cluster
+    FiCaSchedulerApp schedulerApp1 = cs.getApplicationAttempt(
+        am1.getApplicationAttemptId());
+    Assert.assertEquals(7, schedulerApp1.getLiveContainers().size());
+
+    // Submit app2 to queue-c and asks for a 1G container for AM
+    RMApp app2 = rm1.submitApp(1 * GB, "app", "user", null, "c");
+    MockAM am2 = MockRM.launchAndRegisterAM(app2, rm1, nm2);
+
+    // NM1/NM2 has available resource = 0G
+    Assert.assertEquals(0 * GB, cs.getNode(nm1.getNodeId())
+        .getUnallocatedResource().getMemory());
+    Assert.assertEquals(0 * GB, cs.getNode(nm2.getNodeId())
+        .getUnallocatedResource().getMemory());
+
+    // AM asks for a 1 * GB container with unknown host and unknown rack
+    am2.allocate(Arrays.asList(ResourceRequest
+        .newInstance(Priority.newInstance(1), ResourceRequest.ANY,
+            Resources.createResource(1 * GB), 1), ResourceRequest
+        .newInstance(Priority.newInstance(1), "unknownhost",
+            Resources.createResource(1 * GB), 1), ResourceRequest
+        .newInstance(Priority.newInstance(1), "/default-rack",
+            Resources.createResource(1 * GB), 1)), null);
+
+    // Get edit policy and do one update
+    SchedulingEditPolicy editPolicy = getSchedulingEditPolicy(rm1);
+
+    // Call edit schedule twice, and check if one container from app1 marked
+    // to be "killable"
+    editPolicy.editSchedule();
+    editPolicy.editSchedule();
+
+    PreemptionManager pm = cs.getPreemptionManager();
+    Map<ContainerId, RMContainer> killableContainers =
+        waitKillableContainersSize(pm, "a", RMNodeLabelsManager.NO_LABEL, 1);
+    Assert.assertEquals(killableContainers.entrySet().iterator().next().getKey()
+        .getApplicationAttemptId(), am1.getApplicationAttemptId());
+
+    // Call CS.handle once to see if container preempted
+    cs.handle(new NodeUpdateSchedulerEvent(rmNode2));
+
+    FiCaSchedulerApp schedulerApp2 = cs.getApplicationAttempt(
+        am2.getApplicationAttemptId());
+
+    // App1 has 7 containers, and app2 has 1 containers (no container preempted)
+    Assert.assertEquals(7, schedulerApp1.getLiveContainers().size());
+    Assert.assertEquals(1, schedulerApp2.getLiveContainers().size());
+
+    // Do allocation again, one container will be preempted
+    cs.handle(new NodeUpdateSchedulerEvent(rmNode2));
+
+    // App1 has 6 containers, and app2 has 2 containers (new container allocated)
+    Assert.assertEquals(6, schedulerApp1.getLiveContainers().size());
+    Assert.assertEquals(2, schedulerApp2.getLiveContainers().size());
+
+    rm1.close();
+  }
+
+  @Test (timeout = 60000)
+  public void testPreemptionConsidersHardNodeLocality()
+      throws Exception {
+    /**
+     * Test case: same as testSimplePreemption steps 1-3.
+     *
+     * Step 4: app2 asks for 1G container with hard locality specified, and
+     *         asked host is not existed
+     * Confirm system doesn't preempt any container.
+     */
+    MockRM rm1 = new MockRM(conf);
+    rm1.getRMContext().setNodeLabelManager(mgr);
+    rm1.start();
+    MockNM nm1 = rm1.registerNode("h1:1234", 4 * GB);
+    MockNM nm2 = rm1.registerNode("h2:1234", 4 * GB);
+    CapacityScheduler cs = (CapacityScheduler) rm1.getResourceScheduler();
+    RMNode rmNode1 = rm1.getRMContext().getRMNodes().get(nm1.getNodeId());
+    RMNode rmNode2 = rm1.getRMContext().getRMNodes().get(nm2.getNodeId());
+
+    // launch an app to queue, AM container should be launched in nm1
+    RMApp app1 = rm1.submitApp(1 * GB, "app", "user", null, "a");
+    MockAM am1 = MockRM.launchAndRegisterAM(app1, rm1, nm1);
+
+    am1.allocate("*", 1 * GB, 6, new ArrayList<ContainerId>());
+
+    // Do allocation 3 times for node1/node2
+    for (int i = 0; i < 3; i++) {
+      cs.handle(new NodeUpdateSchedulerEvent(rmNode1));
+    }
+    for (int i = 0; i < 3; i++) {
+      cs.handle(new NodeUpdateSchedulerEvent(rmNode2));
+    }
+
+    // App1 should have 7 containers now, and no available resource for cluster
+    FiCaSchedulerApp schedulerApp1 = cs.getApplicationAttempt(
+        am1.getApplicationAttemptId());
+    Assert.assertEquals(7, schedulerApp1.getLiveContainers().size());
+
+    // Submit app2 to queue-c and asks for a 1G container for AM
+    RMApp app2 = rm1.submitApp(1 * GB, "app", "user", null, "c");
+    MockAM am2 = MockRM.launchAndRegisterAM(app2, rm1, nm2);
+
+    // NM1/NM2 has available resource = 0G
+    Assert.assertEquals(0 * GB, cs.getNode(nm1.getNodeId())
+        .getUnallocatedResource().getMemory());
+    Assert.assertEquals(0 * GB, cs.getNode(nm2.getNodeId())
+        .getUnallocatedResource().getMemory());
+
+    // AM asks for a 1 * GB container for h3 with hard locality,
+    // h3 doesn't exist in the cluster
+    am2.allocate(Arrays.asList(ResourceRequest
+        .newInstance(Priority.newInstance(1), ResourceRequest.ANY,
+            Resources.createResource(1 * GB), 1, true), ResourceRequest
+        .newInstance(Priority.newInstance(1), "h3",
+            Resources.createResource(1 * GB), 1, false), ResourceRequest
+        .newInstance(Priority.newInstance(1), "/default-rack",
+            Resources.createResource(1 * GB), 1, false)), null);
+
+    // Get edit policy and do one update
+    SchedulingEditPolicy editPolicy = getSchedulingEditPolicy(rm1);
+
+    // Call edit schedule twice, and check if one container from app1 marked
+    // to be "killable"
+    editPolicy.editSchedule();
+    editPolicy.editSchedule();
+
+    PreemptionManager pm = cs.getPreemptionManager();
+    Map<ContainerId, RMContainer> killableContainers =
+        waitKillableContainersSize(pm, "a", RMNodeLabelsManager.NO_LABEL, 1);
+    Assert.assertEquals(killableContainers.entrySet().iterator().next().getKey()
+        .getApplicationAttemptId(), am1.getApplicationAttemptId());
+
+    // Call CS.handle once to see if container preempted
+    cs.handle(new NodeUpdateSchedulerEvent(rmNode2));
+
+    FiCaSchedulerApp schedulerApp2 = cs.getApplicationAttempt(
+        am2.getApplicationAttemptId());
+
+    // App1 has 7 containers, and app2 has 1 containers (no container preempted)
+    Assert.assertEquals(7, schedulerApp1.getLiveContainers().size());
+    Assert.assertEquals(1, schedulerApp2.getLiveContainers().size());
+
+    // Do allocation again, nothing will be preempted
+    cs.handle(new NodeUpdateSchedulerEvent(rmNode2));
+
+    // App1 has 7 containers, and app2 has 1 containers (no container allocated)
+    Assert.assertEquals(7, schedulerApp1.getLiveContainers().size());
+    Assert.assertEquals(1, schedulerApp2.getLiveContainers().size());
+
+    rm1.close();
+  }
+
+  @Test (timeout = 60000)
+  public void testPreemptionPolicyShouldRespectAlreadyMarkedKillableContainers()
+      throws Exception {
+    /**
+     * Test case:
+     * <pre>
+     *             Root
+     *            /  |  \
+     *           a   b   c
+     *          10   20  70
+     * </pre>
+     * Submit applications to two queues, one uses more than the other, so
+     * preemption will happen.
+     *
+     * Check:
+     * 1) Killable containers resources will be excluded from PCPP (no duplicated
+     *    container added to killable list)
+     * 2) When more resources need to be preempted, new containers will be selected
+     *    and killable containers will be considered
+     */
+    MockRM rm1 = new MockRM(conf);
+    rm1.getRMContext().setNodeLabelManager(mgr);
+    rm1.start();
+    MockNM nm1 = rm1.registerNode("h1:1234", 8 * GB);
+    CapacityScheduler cs = (CapacityScheduler) rm1.getResourceScheduler();
+    RMNode rmNode1 = rm1.getRMContext().getRMNodes().get(nm1.getNodeId());
+
+    // launch an app to queue, AM container should be launched in nm1
+    RMApp app1 = rm1.submitApp(1 * GB, "app", "user", null, "a");
+    MockAM am1 = MockRM.launchAndRegisterAM(app1, rm1, nm1);
+
+    am1.allocate("*", 1 * GB, 6, new ArrayList<ContainerId>());
+
+    // Do allocation 6 times for node1
+    for (int i = 0; i < 6; i++) {
+      cs.handle(new NodeUpdateSchedulerEvent(rmNode1));
+    }
+
+    // App1 should have 7 containers now, and no available resource for cluster
+    FiCaSchedulerApp schedulerApp1 = cs.getApplicationAttempt(
+        am1.getApplicationAttemptId());
+    Assert.assertEquals(7, schedulerApp1.getLiveContainers().size());
+
+    // Submit app2 to queue-c and asks for a 1G container for AM
+    RMApp app2 = rm1.submitApp(1 * GB, "app", "user", null, "c");
+    MockAM am2 = MockRM.launchAndRegisterAM(app2, rm1, nm1);
+
+    // NM1 has available resource = 0G
+    Assert.assertEquals(0 * GB, cs.getNode(nm1.getNodeId())
+        .getUnallocatedResource().getMemory());
+    am2.allocate("*", 1 * GB, 1, new ArrayList<ContainerId>());
+
+    // Get edit policy and do one update
+    ProportionalCapacityPreemptionPolicy editPolicy =
+        (ProportionalCapacityPreemptionPolicy) getSchedulingEditPolicy(rm1);
+
+    // Call edit schedule twice, and check if one container from app1 marked
+    // to be "killable"
+    editPolicy.editSchedule();
+    editPolicy.editSchedule();
+
+    PreemptionManager pm = cs.getPreemptionManager();
+    waitKillableContainersSize(pm, "a", RMNodeLabelsManager.NO_LABEL, 1);
+
+    // Check killable containers and to-be-preempted containers in edit policy
+    Assert.assertEquals(0, editPolicy.getToPreemptContainers().size());
+
+    // Run edit schedule again, confirm status doesn't changed
+    editPolicy.editSchedule();
+    Assert.assertEquals(0, editPolicy.getToPreemptContainers().size());
+
+    // Save current to kill containers
+    Set<ContainerId> previousKillableContainers = new HashSet<>(
+        pm.getKillableContainersMap("a", RMNodeLabelsManager.NO_LABEL)
+            .keySet());
+
+    // Update request resource of c from 1 to 2, so we need to preempt
+    // one more container
+    am2.allocate("*", 1 * GB, 2, new ArrayList<ContainerId>());
+
+    // Call editPolicy.editSchedule() once, we should have 1 container in to-preempt map
+    // and 1 container in killable map
+    editPolicy.editSchedule();
+    Assert.assertEquals(1, editPolicy.getToPreemptContainers().size());
+
+    // Call editPolicy.editSchedule() once more, we should have 2 containers killable map
+    editPolicy.editSchedule();
+    Assert.assertEquals(0, editPolicy.getToPreemptContainers().size());
+
+    // Check if previous killable containers included by new killable containers
+    Map<ContainerId, RMContainer> killableContainers =
+        waitKillableContainersSize(pm, "a", RMNodeLabelsManager.NO_LABEL, 2);
+    Assert.assertTrue(
+        Sets.difference(previousKillableContainers, killableContainers.keySet())
+            .isEmpty());
+  }
+
+  @Test (timeout = 60000)
+  public void testPreemptionPolicyCleanupKillableContainersWhenNoPreemptionNeeded()
+      throws Exception {
+    /**
+     * Test case:
+     * <pre>
+     *             Root
+     *            /  |  \
+     *           a   b   c
+     *          10   20  70
+     * </pre>
+     * Submit applications to two queues, one uses more than the other, so
+     * preemption will happen.
+     *
+     * Check:
+     * 1) Containers will be marked to killable
+     * 2) Cancel resource request
+     * 3) Killable containers will be cancelled from policy and scheduler
+     */
+    MockRM rm1 = new MockRM(conf);
+    rm1.getRMContext().setNodeLabelManager(mgr);
+    rm1.start();
+    MockNM nm1 = rm1.registerNode("h1:1234", 8 * GB);
+    CapacityScheduler cs = (CapacityScheduler) rm1.getResourceScheduler();
+    RMNode rmNode1 = rm1.getRMContext().getRMNodes().get(nm1.getNodeId());
+
+    // launch an app to queue, AM container should be launched in nm1
+    RMApp app1 = rm1.submitApp(1 * GB, "app", "user", null, "a");
+    MockAM am1 = MockRM.launchAndRegisterAM(app1, rm1, nm1);
+
+    am1.allocate("*", 1 * GB, 6, new ArrayList<ContainerId>());
+
+    // Do allocation 6 times for node1
+    for (int i = 0; i < 6; i++) {
+      cs.handle(new NodeUpdateSchedulerEvent(rmNode1));
+    }
+
+    // App1 should have 7 containers now, and no available resource for cluster
+    FiCaSchedulerApp schedulerApp1 = cs.getApplicationAttempt(
+        am1.getApplicationAttemptId());
+    Assert.assertEquals(7, schedulerApp1.getLiveContainers().size());
+
+    // Submit app2 to queue-c and asks for a 1G container for AM
+    RMApp app2 = rm1.submitApp(1 * GB, "app", "user", null, "c");
+    MockAM am2 = MockRM.launchAndRegisterAM(app2, rm1, nm1);
+
+    // NM1 has available resource = 0G
+    Assert.assertEquals(0 * GB, cs.getNode(nm1.getNodeId())
+        .getUnallocatedResource().getMemory());
+    am2.allocate("*", 3 * GB, 1, new ArrayList<ContainerId>());
+
+    // Get edit policy and do one update
+    ProportionalCapacityPreemptionPolicy editPolicy =
+        (ProportionalCapacityPreemptionPolicy) getSchedulingEditPolicy(rm1);
+
+    // Call edit schedule twice, and check if 3 container from app1 marked
+    // to be "killable"
+    editPolicy.editSchedule();
+    editPolicy.editSchedule();
+
+    PreemptionManager pm = cs.getPreemptionManager();
+    waitKillableContainersSize(pm, "a", RMNodeLabelsManager.NO_LABEL, 3);
+
+    // Change reqeust from 3G to 2G, now we can preempt one less container. (3->2)
+    am2.allocate("*", 2 * GB, 1, new ArrayList<ContainerId>());
+    editPolicy.editSchedule();
+    Assert.assertEquals(0, editPolicy.getToPreemptContainers().size());
+    waitKillableContainersSize(pm, "a", RMNodeLabelsManager.NO_LABEL, 2);
+
+    // Call editSchedule once more to make sure still nothing happens
+    editPolicy.editSchedule();
+    Assert.assertEquals(0, editPolicy.getToPreemptContainers().size());
+    waitKillableContainersSize(pm, "a", RMNodeLabelsManager.NO_LABEL, 2);
+  }
+
+  @Test (timeout = 60000)
+  public void testPreemptionConsidersUserLimit()
+      throws Exception {
+    /**
+     * Test case: Submit two application (app1/app2) to different queues, queue
+     * structure:
+     *
+     * <pre>
+     *             Root
+     *            /  |  \
+     *           a   b   c
+     *          10   20  70
+     * </pre>
+     *
+     * Queue-c's user-limit-factor = 0.1, so single user cannot allocate >1 containers in queue-c
+     *
+     * 1) Two nodes in the cluster, each of them has 4G.
+     *
+     * 2) app1 submit to queue-a first, it asked 7 * 1G containers, so there's no
+     * more resource available.
+     *
+     * 3) app2 submit to queue-c, ask for one 1G container (for AM)
+     *
+     * Now the cluster is fulfilled.
+     *
+     * 4) app2 asks for another 1G container, system will preempt one container
+     * from app1, and app2 will receive the preempted container
+     */
+    CapacitySchedulerConfiguration csConf = new CapacitySchedulerConfiguration(conf);
+    csConf.setUserLimitFactor(CapacitySchedulerConfiguration.ROOT + ".c", 0.1f);
+    MockRM rm1 = new MockRM(csConf);
+    rm1.getRMContext().setNodeLabelManager(mgr);
+    rm1.start();
+    MockNM nm1 = rm1.registerNode("h1:1234", 4 * GB);
+    MockNM nm2 = rm1.registerNode("h2:1234", 4 * GB);
+    CapacityScheduler cs = (CapacityScheduler) rm1.getResourceScheduler();
+    RMNode rmNode1 = rm1.getRMContext().getRMNodes().get(nm1.getNodeId());
+    RMNode rmNode2 = rm1.getRMContext().getRMNodes().get(nm2.getNodeId());
+
+    // launch an app to queue, AM container should be launched in nm1
+    RMApp app1 = rm1.submitApp(1 * GB, "app", "user", null, "a");
+    MockAM am1 = MockRM.launchAndRegisterAM(app1, rm1, nm1);
+
+    am1.allocate("*", 1 * GB, 6, new ArrayList<ContainerId>());
+
+    // Do allocation 3 times for node1/node2
+    for (int i = 0; i < 3; i++) {
+      cs.handle(new NodeUpdateSchedulerEvent(rmNode1));
+      cs.handle(new NodeUpdateSchedulerEvent(rmNode2));
+    }
+
+    // App1 should have 7 containers now, and no available resource for cluster
+    FiCaSchedulerApp schedulerApp1 = cs.getApplicationAttempt(
+        am1.getApplicationAttemptId());
+    Assert.assertEquals(7, schedulerApp1.getLiveContainers().size());
+
+    // Submit app2 to queue-c and asks for a 1G container for AM
+    RMApp app2 = rm1.submitApp(1 * GB, "app", "user", null, "c");
+    MockAM am2 = MockRM.launchAndRegisterAM(app2, rm1, nm2);
+
+    // NM1/NM2 has available resource = 0G
+    Assert.assertEquals(0 * GB, cs.getNode(nm1.getNodeId())
+        .getUnallocatedResource().getMemory());
+    Assert.assertEquals(0 * GB, cs.getNode(nm2.getNodeId())
+        .getUnallocatedResource().getMemory());
+
+    // AM asks for a 1 * GB container
+    am2.allocate(Arrays.asList(ResourceRequest
+        .newInstance(Priority.newInstance(1), ResourceRequest.ANY,
+            Resources.createResource(1 * GB), 1)), null);
+
+    // Get edit policy and do one update
+    SchedulingEditPolicy editPolicy = getSchedulingEditPolicy(rm1);
+
+    // Call edit schedule twice, and check if no container from app1 marked
+    // to be "killable"
+    editPolicy.editSchedule();
+    editPolicy.editSchedule();
+
+    // No preemption happens
+    PreemptionManager pm = cs.getPreemptionManager();
+    Map<ContainerId, RMContainer> killableContainers =
+        waitKillableContainersSize(pm, "a", RMNodeLabelsManager.NO_LABEL, 0);
+    Assert.assertEquals(0, killableContainers.size());
+
+    // Call CS.handle once to see if container preempted
+    cs.handle(new NodeUpdateSchedulerEvent(rmNode2));
+
+    FiCaSchedulerApp schedulerApp2 = cs.getApplicationAttempt(
+        am2.getApplicationAttemptId());
+
+    // App1 has 7 containers, and app2 has 1 containers (nothing preempted)
+    Assert.assertEquals(7, schedulerApp1.getLiveContainers().size());
+    Assert.assertEquals(1, schedulerApp2.getLiveContainers().size());
+
+    rm1.close();
+  }
+
+  private Map<ContainerId, RMContainer> waitKillableContainersSize(
+      PreemptionManager pm, String queueName, String partition,
+      int expectedSize) throws InterruptedException {
+    Map<ContainerId, RMContainer> killableContainers =
+        pm.getKillableContainersMap(queueName, partition);
+
+    int wait = 0;
+    // Wait for at most 5 sec (it should be super fast actually)
+    while (expectedSize != killableContainers.size() && wait < 500) {
+      killableContainers = pm.getKillableContainersMap(queueName, partition);
+      Thread.sleep(10);
+      wait++;
+    }
+
+    Assert.assertEquals(expectedSize, killableContainers.size());
+    return killableContainers;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7e8c9beb/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 5169337..1612201 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
@@ -51,6 +51,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerEven
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerImpl;
 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.capacity.preemption.PreemptionManager;
 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.server.utils.BuilderUtils;
@@ -99,6 +100,7 @@ public class TestChildQueueOrder {
     when(csContext.getResourceCalculator()).
     thenReturn(resourceComparator);
     when(csContext.getRMContext()).thenReturn(rmContext);
+    when(csContext.getPreemptionManager()).thenReturn(new PreemptionManager());
   }
 
   private FiCaSchedulerApp getMockApplication(int appId, String user) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7e8c9beb/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 69b0813..87a3d51 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
@@ -71,6 +71,7 @@ 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.QueueMetrics;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceLimits;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.preemption.PreemptionManager;
 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.server.resourcemanager.scheduler.event.AppAddedSchedulerEvent;
@@ -150,6 +151,7 @@ public class TestLeafQueue {
         thenReturn(CapacityScheduler.nonPartitionedQueueComparator);
     when(csContext.getResourceCalculator()).
         thenReturn(resourceCalculator);
+    when(csContext.getPreemptionManager()).thenReturn(new PreemptionManager());
     when(csContext.getRMContext()).thenReturn(rmContext);
     RMContainerTokenSecretManager containerTokenSecretManager =
         new RMContainerTokenSecretManager(conf);
@@ -3092,6 +3094,7 @@ public class TestLeafQueue {
         Resources.createResource(GB, 1));
     when(csContext.getMaximumResourceCapability()).thenReturn(
         Resources.createResource(2 * GB, 2));
+    when(csContext.getPreemptionManager()).thenReturn(new PreemptionManager());
     return csContext;
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7e8c9beb/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 bbf6e43..1ee201d 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
@@ -39,6 +39,7 @@ 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.recovery.MemoryRMStateStore;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptState;
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
@@ -1676,4 +1677,100 @@ public class TestNodeLabelContainerAllocation {
     checkNumOfContainersInAnAppOnGivenNode(2, nm1.getNodeId(),
         cs.getApplicationAttempt(am1.getApplicationAttemptId()));
   }
+
+  @Test
+  public void testParentQueueMaxCapsAreRespected() throws Exception {
+    /*
+     * Queue tree:
+     *          Root
+     *        /     \
+     *       A       B
+     *      / \
+     *     A1 A2
+     *
+     * A has 50% capacity and 50% max capacity (of label=x)
+     * A1/A2 has 50% capacity and 100% max capacity (of label=x)
+     * Cluster has one node (label=x) with resource = 24G.
+     * So we can at most use 12G resources under queueA.
+     */
+    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, 10);
+    csConf.setAccessibleNodeLabels(A, toSet("x"));
+    csConf.setCapacityByLabel(A, "x", 50);
+    csConf.setMaximumCapacityByLabel(A, "x", 50);
+
+    final String B = CapacitySchedulerConfiguration.ROOT + ".b";
+    csConf.setCapacity(B, 90);
+    csConf.setAccessibleNodeLabels(B, toSet("x"));
+    csConf.setCapacityByLabel(B, "x", 50);
+    csConf.setMaximumCapacityByLabel(B, "x", 50);
+
+    // Define 2nd-level queues
+    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);
+    csConf.setMaximumCapacityByLabel(A1, "x", 100);
+    csConf.setUserLimitFactor(A1, 100.0f);
+
+    final String A2 = A + ".a2";
+    csConf.setCapacity(A2, 50);
+    csConf.setAccessibleNodeLabels(A2, toSet("x"));
+    csConf.setCapacityByLabel(A2, "x", 50);
+    csConf.setMaximumCapacityByLabel(A2, "x", 100);
+    csConf.setUserLimitFactor(A2, 100.0f);
+
+    // set node -> label
+    mgr.addToCluserNodeLabels(ImmutableSet.of(
+        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 =
+        new MockNM("h1:1234", 24 * GB, rm.getResourceTrackerService());
+    nm1.registerNode();
+
+    // Launch app1 in a1, resource usage is 1GB (am) + 4GB * 2 = 9GB
+    RMApp app1 = rm.submitApp(1 * GB, "app", "user", null, "a1", "x");
+    MockAM am1 = MockRM.launchAndRegisterAM(app1, rm, nm1);
+    am1.allocate("*", 4 * GB, 2, new ArrayList<ContainerId>(), "x");
+    doNMHeartbeat(rm, nm1.getNodeId(), 10);
+    checkNumOfContainersInAnAppOnGivenNode(3, nm1.getNodeId(),
+        cs.getApplicationAttempt(am1.getApplicationAttemptId()));
+
+    // Try to launch app2 in a2, asked 2GB, should success
+    RMApp app2 = rm.submitApp(2 * GB, "app", "user", null, "a2", "x");
+    MockAM am2 = MockRM.launchAndRegisterAM(app2, rm, nm1);
+
+    // am2 asks more resources, cannot success because current used = 9G (app1)
+    // + 2G (app2) = 11G, and queue's max capacity = 12G
+    am2.allocate("*", 2 * GB, 2, new ArrayList<ContainerId>(), "x");
+
+    doNMHeartbeat(rm, nm1.getNodeId(), 10);
+    checkNumOfContainersInAnAppOnGivenNode(1, nm1.getNodeId(),
+        cs.getApplicationAttempt(am2.getApplicationAttemptId()));
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7e8c9beb/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 f73baa4..23dc860 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
@@ -47,6 +47,7 @@ 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.capacity.preemption.PreemptionManager;
 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.DefaultResourceCalculator;
@@ -92,6 +93,7 @@ public class TestParentQueue {
         thenReturn(Resources.createResource(100 * 16 * GB, 100 * 32));
     when(csContext.getNonPartitionedQueueComparator()).
     thenReturn(CapacityScheduler.nonPartitionedQueueComparator);
+    when(csContext.getPreemptionManager()).thenReturn(new PreemptionManager());
     when(csContext.getResourceCalculator()).
     thenReturn(resourceComparator);
     when(csContext.getRMContext()).thenReturn(rmContext);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7e8c9beb/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 2ef5e39..56facee 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
@@ -55,6 +55,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerImpl
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ActiveUsersManager;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceLimits;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplicationAttempt.AMState;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.preemption.PreemptionManager;
 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.server.resourcemanager.security.RMContainerTokenSecretManager;
@@ -126,6 +127,7 @@ public class TestReservations {
     when(csContext.getNonPartitionedQueueComparator()).thenReturn(
         CapacityScheduler.nonPartitionedQueueComparator);
     when(csContext.getResourceCalculator()).thenReturn(resourceCalculator);
+    when(csContext.getPreemptionManager()).thenReturn(new PreemptionManager());
     when(csContext.getRMContext()).thenReturn(rmContext);
     RMContainerTokenSecretManager containerTokenSecretManager = new RMContainerTokenSecretManager(
         conf);