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 2017/09/07 21:36:14 UTC

[01/37] hadoop git commit: Plan/ResourceAllocation data structure enhancements required to support recurring reservations in ReservationSystem.

Repository: hadoop
Updated Branches:
  refs/heads/HDFS-7240 5cd6d7bb8 -> e319be93b


http://git-wip-us.apache.org/repos/asf/hadoop/blob/7996eca7/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestInMemoryPlan.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/reservation/TestInMemoryPlan.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestInMemoryPlan.java
index bc98e2f..03569d4 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestInMemoryPlan.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestInMemoryPlan.java
@@ -40,6 +40,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.reservation.planning.Planne
 import org.apache.hadoop.yarn.server.resourcemanager.reservation.planning.ReservationAgent;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueMetrics;
 import org.apache.hadoop.yarn.util.Clock;
+import org.apache.hadoop.yarn.util.UTCClock;
 import org.apache.hadoop.yarn.util.resource.DefaultResourceCalculator;
 import org.apache.hadoop.yarn.util.resource.ResourceCalculator;
 import org.junit.After;
@@ -47,6 +48,10 @@ import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
 
+/**
+ * Testing the class {@link InMemoryPlan}.
+ */
+@SuppressWarnings("checkstyle:nowhitespaceafter")
 public class TestInMemoryPlan {
 
   private String user = "yarn";
@@ -62,6 +67,7 @@ public class TestInMemoryPlan {
   private ReservationAgent agent;
   private Planner replanner;
   private RMContext context;
+  private long maxPeriodicity;
 
   @Before
   public void setUp() throws PlanningException {
@@ -72,7 +78,7 @@ public class TestInMemoryPlan {
 
     clock = mock(Clock.class);
     queueMetrics = mock(QueueMetrics.class);
-    policy = mock(SharingPolicy.class);
+    policy = new NoOverCommitPolicy();
     replanner = mock(Planner.class);
 
     when(clock.getTime()).thenReturn(1L);
@@ -95,15 +101,41 @@ public class TestInMemoryPlan {
 
   @Test
   public void testAddReservation() {
-    Plan plan =
-        new InMemoryPlan(queueMetrics, policy, agent, totalCapacity, 1L,
-            resCalc, minAlloc, maxAlloc, planName, replanner, true, context);
+    Plan plan = new InMemoryPlan(queueMetrics, policy, agent, totalCapacity, 1L,
+        resCalc, minAlloc, maxAlloc, planName, replanner, true, context);
     ReservationId reservationID =
-            ReservationSystemTestUtil.getNewReservationId();
+        ReservationSystemTestUtil.getNewReservationId();
     int[] alloc = { 10, 10, 10, 10, 10, 10 };
     int start = 100;
-    ReservationAllocation rAllocation = createReservationAllocation
-            (reservationID, start, alloc);
+    ReservationAllocation rAllocation =
+        createReservationAllocation(reservationID, start, alloc);
+    Assert.assertNull(plan.getReservationById(reservationID));
+    try {
+      plan.addReservation(rAllocation, false);
+    } catch (PlanningException e) {
+      Assert.fail(e.getMessage());
+    }
+    doAssertions(plan, rAllocation);
+    checkAllocation(plan, alloc, start, 0);
+  }
+
+  @Test
+  public void testAddPeriodicReservation() throws PlanningException {
+
+    maxPeriodicity = 100;
+    Plan plan = new InMemoryPlan(queueMetrics, policy, agent, totalCapacity, 1L,
+        resCalc, minAlloc, maxAlloc, planName, replanner, true, maxPeriodicity,
+        context, new UTCClock());
+
+    ReservationId reservationID =
+        ReservationSystemTestUtil.getNewReservationId();
+    int[] alloc = { 10, 50 };
+    int start = 10;
+    long period = 20;
+    ReservationAllocation rAllocation = createReservationAllocation(
+        reservationID, start, alloc, String.valueOf(period));
+    // use periodicity of 1hr
+    rAllocation.setPeriodicity(period);
     Assert.assertNull(plan.getReservationById(reservationID));
     try {
       plan.addReservation(rAllocation, false);
@@ -111,32 +143,54 @@ public class TestInMemoryPlan {
       Assert.fail(e.getMessage());
     }
     doAssertions(plan, rAllocation);
-    checkAllocation(plan, alloc, start);
+    checkAllocation(plan, alloc, start, period);
+
+    RLESparseResourceAllocation available =
+        plan.getAvailableResourceOverTime(user, reservationID, 150, 330, 50);
+    System.out.println(available);
   }
 
-  private void checkAllocation(Plan plan, int[] alloc, int start) {
+  private void checkAllocation(Plan plan, int[] alloc, int start,
+      long periodicity) {
+    long end = start + alloc.length;
+    if (periodicity > 0) {
+      end = end + maxPeriodicity;
+    }
     RLESparseResourceAllocation userCons =
-        plan.getConsumptionForUserOverTime(user, start, start + alloc.length);
+        plan.getConsumptionForUserOverTime(user, start, end * 3);
 
     for (int i = 0; i < alloc.length; i++) {
-      Assert.assertEquals(Resource.newInstance(1024 * (alloc[i]), (alloc[i])),
-          plan.getTotalCommittedResources(start + i));
-      Assert.assertEquals(Resource.newInstance(1024 * (alloc[i]), (alloc[i])),
-          userCons.getCapacityAtTime(start + i));
+      // only one instance for non-periodic reservation
+      if (periodicity <= 0) {
+        Assert.assertEquals(Resource.newInstance(1024 * (alloc[i]), (alloc[i])),
+            plan.getTotalCommittedResources(start + i));
+        Assert.assertEquals(Resource.newInstance(1024 * (alloc[i]), (alloc[i])),
+            userCons.getCapacityAtTime(start + i));
+      } else {
+        // periodic reservations should repeat
+        long y = 0;
+        Resource res = Resource.newInstance(1024 * (alloc[i]), (alloc[i]));
+        while (y <= end * 2) {
+          Assert.assertEquals("At time: " + start + i + y, res,
+              plan.getTotalCommittedResources(start + i + y));
+          Assert.assertEquals(" At time: " + (start + i + y), res,
+              userCons.getCapacityAtTime(start + i + y));
+          y = y + periodicity;
+        }
+      }
     }
   }
 
   @Test
   public void testAddEmptyReservation() {
-    Plan plan =
-        new InMemoryPlan(queueMetrics, policy, agent, totalCapacity, 1L,
-            resCalc, minAlloc, maxAlloc, planName, replanner, true, context);
+    Plan plan = new InMemoryPlan(queueMetrics, policy, agent, totalCapacity, 1L,
+        resCalc, minAlloc, maxAlloc, planName, replanner, true, context);
     ReservationId reservationID =
-            ReservationSystemTestUtil.getNewReservationId();
+        ReservationSystemTestUtil.getNewReservationId();
     int[] alloc = {};
     int start = 100;
-    ReservationAllocation rAllocation = createReservationAllocation
-            (reservationID, start, alloc);
+    ReservationAllocation rAllocation =
+        createReservationAllocation(reservationID, start, alloc);
     Assert.assertNull(plan.getReservationById(reservationID));
     try {
       plan.addReservation(rAllocation, false);
@@ -148,15 +202,14 @@ public class TestInMemoryPlan {
   @Test
   public void testAddReservationAlreadyExists() {
     // First add a reservation
-    Plan plan =
-        new InMemoryPlan(queueMetrics, policy, agent, totalCapacity, 1L,
-            resCalc, minAlloc, maxAlloc, planName, replanner, true, context);
+    Plan plan = new InMemoryPlan(queueMetrics, policy, agent, totalCapacity, 1L,
+        resCalc, minAlloc, maxAlloc, planName, replanner, true, context);
     ReservationId reservationID =
-            ReservationSystemTestUtil.getNewReservationId();
+        ReservationSystemTestUtil.getNewReservationId();
     int[] alloc = { 10, 10, 10, 10, 10, 10 };
     int start = 100;
-    ReservationAllocation rAllocation = createReservationAllocation
-            (reservationID, start, alloc);
+    ReservationAllocation rAllocation =
+        createReservationAllocation(reservationID, start, alloc);
     Assert.assertNull(plan.getReservationById(reservationID));
     try {
       plan.addReservation(rAllocation, false);
@@ -164,7 +217,7 @@ public class TestInMemoryPlan {
       Assert.fail(e.getMessage());
     }
     doAssertions(plan, rAllocation);
-    checkAllocation(plan, alloc, start);
+    checkAllocation(plan, alloc, start, 0);
 
     // Try to add it again
     try {
@@ -180,16 +233,15 @@ public class TestInMemoryPlan {
 
   @Test
   public void testUpdateReservation() {
-    Plan plan =
-        new InMemoryPlan(queueMetrics, policy, agent, totalCapacity, 1L,
-            resCalc, minAlloc, maxAlloc, planName, replanner, true, context);
+    Plan plan = new InMemoryPlan(queueMetrics, policy, agent, totalCapacity, 1L,
+        resCalc, minAlloc, maxAlloc, planName, replanner, true, context);
     ReservationId reservationID =
         ReservationSystemTestUtil.getNewReservationId();
     // First add a reservation
     int[] alloc = { 10, 10, 10, 10, 10, 10 };
     int start = 100;
-    ReservationAllocation rAllocation = createReservationAllocation
-            (reservationID, start, alloc);
+    ReservationAllocation rAllocation =
+        createReservationAllocation(reservationID, start, alloc);
     Assert.assertNull(plan.getReservationById(reservationID));
     try {
       plan.addReservation(rAllocation, false);
@@ -210,8 +262,8 @@ public class TestInMemoryPlan {
     // Now update it
     start = 110;
     int[] updatedAlloc = { 0, 5, 10, 10, 5, 0 };
-    rAllocation = createReservationAllocation(reservationID, start,
-            updatedAlloc, true);
+    rAllocation =
+        createReservationAllocation(reservationID, start, updatedAlloc, true);
     try {
       plan.updateReservation(rAllocation);
     } catch (PlanningException e) {
@@ -219,32 +271,71 @@ public class TestInMemoryPlan {
     }
     doAssertions(plan, rAllocation);
 
-    userCons =
-        plan.getConsumptionForUserOverTime(user, start, start
-            + updatedAlloc.length);
+    userCons = plan.getConsumptionForUserOverTime(user, start,
+        start + updatedAlloc.length);
 
     for (int i = 0; i < updatedAlloc.length; i++) {
-      Assert.assertEquals(
-     Resource.newInstance(1024 * (updatedAlloc[i] + i), updatedAlloc[i]
-              + i), plan.getTotalCommittedResources(start + i));
-      Assert.assertEquals(
-          Resource.newInstance(1024 * (updatedAlloc[i] + i), updatedAlloc[i]
-              + i), userCons.getCapacityAtTime(start + i));
+      Assert.assertEquals(Resource.newInstance(1024 * (updatedAlloc[i] + i),
+          updatedAlloc[i] + i), plan.getTotalCommittedResources(start + i));
+      Assert.assertEquals(Resource.newInstance(1024 * (updatedAlloc[i] + i),
+          updatedAlloc[i] + i), userCons.getCapacityAtTime(start + i));
+    }
+  }
+
+  @Test
+  public void testUpdatePeriodicReservation() {
+    Plan plan = new InMemoryPlan(queueMetrics, policy, agent, totalCapacity, 1L,
+        resCalc, minAlloc, maxAlloc, planName, replanner, true, context);
+    // First add a reservation
+    ReservationId reservationID =
+        ReservationSystemTestUtil.getNewReservationId();
+    int[] alloc = { 10, 20 };
+    int start = 100;
+    ReservationAllocation rAllocation =
+        createReservationAllocation(reservationID, start, alloc);
+    // use periodicity of 1hr
+    long period = 3600000;
+    rAllocation.getReservationDefinition()
+        .setRecurrenceExpression(String.valueOf(period));
+    rAllocation.setPeriodicity(period);
+    Assert.assertNull(plan.getReservationById(reservationID));
+    try {
+      plan.addReservation(rAllocation, false);
+    } catch (PlanningException e) {
+      Assert.fail(e.getMessage());
+    }
+    System.out.println(plan.toString());
+    doAssertions(plan, rAllocation);
+    checkAllocation(plan, alloc, start, period);
+
+    // Now update it
+    start = 110;
+    int[] updatedAlloc = { 30, 40, 50 };
+    rAllocation =
+        createReservationAllocation(reservationID, start, updatedAlloc);
+    rAllocation.getReservationDefinition()
+        .setRecurrenceExpression(String.valueOf(period));
+    rAllocation.setPeriodicity(period);
+    try {
+      plan.updateReservation(rAllocation);
+    } catch (PlanningException e) {
+      Assert.fail(e.getMessage());
     }
+    doAssertions(plan, rAllocation);
+    checkAllocation(plan, updatedAlloc, start, period);
   }
 
   @Test
   public void testUpdateNonExistingReservation() {
-    Plan plan =
-        new InMemoryPlan(queueMetrics, policy, agent, totalCapacity, 1L,
-            resCalc, minAlloc, maxAlloc, planName, replanner, true, context);
+    Plan plan = new InMemoryPlan(queueMetrics, policy, agent, totalCapacity, 1L,
+        resCalc, minAlloc, maxAlloc, planName, replanner, true, context);
     ReservationId reservationID =
         ReservationSystemTestUtil.getNewReservationId();
     // Try to update a reservation without adding
     int[] alloc = { 10, 10, 10, 10, 10, 10 };
     int start = 100;
     ReservationAllocation rAllocation =
-            createReservationAllocation(reservationID, start, alloc);
+        createReservationAllocation(reservationID, start, alloc);
     Assert.assertNull(plan.getReservationById(reservationID));
     try {
       plan.updateReservation(rAllocation);
@@ -260,15 +351,14 @@ public class TestInMemoryPlan {
   @Test
   public void testDeleteReservation() {
     // First add a reservation
-    Plan plan =
-        new InMemoryPlan(queueMetrics, policy, agent, totalCapacity, 1L,
-            resCalc, minAlloc, maxAlloc, planName, replanner, true, context);
+    Plan plan = new InMemoryPlan(queueMetrics, policy, agent, totalCapacity, 1L,
+        resCalc, minAlloc, maxAlloc, planName, replanner, true, context);
     ReservationId reservationID =
         ReservationSystemTestUtil.getNewReservationId();
     int[] alloc = { 10, 10, 10, 10, 10, 10 };
     int start = 100;
     ReservationAllocation rAllocation =
-            createReservationAllocation(reservationID, start, alloc, true);
+        createReservationAllocation(reservationID, start, alloc, true);
     Assert.assertNull(plan.getReservationById(reservationID));
     try {
       plan.addReservation(rAllocation, false);
@@ -307,10 +397,46 @@ public class TestInMemoryPlan {
   }
 
   @Test
+  public void testDeletePeriodicReservation() {
+    Plan plan = new InMemoryPlan(queueMetrics, policy, agent, totalCapacity, 1L,
+        resCalc, minAlloc, maxAlloc, planName, replanner, true, context);
+    // First add a reservation
+    ReservationId reservationID =
+        ReservationSystemTestUtil.getNewReservationId();
+    int[] alloc = { 10, 20 };
+    int start = 100;
+    ReservationAllocation rAllocation =
+        createReservationAllocation(reservationID, start, alloc);
+    // use periodicity of 1hr
+    long period = 3600000;
+    rAllocation.getReservationDefinition()
+        .setRecurrenceExpression(String.valueOf(period));
+    rAllocation.setPeriodicity(period);
+    Assert.assertNull(plan.getReservationById(reservationID));
+    try {
+      plan.addReservation(rAllocation, false);
+    } catch (PlanningException e) {
+      Assert.fail(e.getMessage());
+    }
+    System.out.println(plan.toString());
+    doAssertions(plan, rAllocation);
+    checkAllocation(plan, alloc, start, period);
+
+    // Now delete it
+    try {
+      plan.deleteReservation(reservationID);
+    } catch (PlanningException e) {
+      Assert.fail(e.getMessage());
+    }
+    Assert.assertNull(plan.getReservationById(reservationID));
+    System.out.print(plan);
+    checkAllocation(plan, new int[] { 0, 0 }, start, period);
+  }
+
+  @Test
   public void testDeleteNonExistingReservation() {
-    Plan plan =
-        new InMemoryPlan(queueMetrics, policy, agent, totalCapacity, 1L,
-            resCalc, minAlloc, maxAlloc, planName, replanner, true, context);
+    Plan plan = new InMemoryPlan(queueMetrics, policy, agent, totalCapacity, 1L,
+        resCalc, minAlloc, maxAlloc, planName, replanner, true, context);
     ReservationId reservationID =
         ReservationSystemTestUtil.getNewReservationId();
     // Try to delete a reservation without adding
@@ -328,8 +454,9 @@ public class TestInMemoryPlan {
 
   @Test
   public void testArchiveCompletedReservations() {
+    SharingPolicy sharingPolicy = mock(SharingPolicy.class);
     Plan plan =
-        new InMemoryPlan(queueMetrics, policy, agent, totalCapacity, 1L,
+        new InMemoryPlan(queueMetrics, sharingPolicy, agent, totalCapacity, 1L,
             resCalc, minAlloc, maxAlloc, planName, replanner, true, context);
     ReservationId reservationID1 =
         ReservationSystemTestUtil.getNewReservationId();
@@ -337,7 +464,7 @@ public class TestInMemoryPlan {
     int[] alloc1 = { 10, 10, 10, 10, 10, 10 };
     int start = 100;
     ReservationAllocation rAllocation =
-            createReservationAllocation(reservationID1, start, alloc1);
+        createReservationAllocation(reservationID1, start, alloc1);
     Assert.assertNull(plan.getReservationById(reservationID1));
     try {
       plan.addReservation(rAllocation, false);
@@ -345,15 +472,14 @@ public class TestInMemoryPlan {
       Assert.fail(e.getMessage());
     }
     doAssertions(plan, rAllocation);
-    checkAllocation(plan, alloc1, start);
-
+    checkAllocation(plan, alloc1, start, 0);
 
     // Now add another one
     ReservationId reservationID2 =
         ReservationSystemTestUtil.getNewReservationId();
     int[] alloc2 = { 0, 5, 10, 5, 0 };
     rAllocation =
-            createReservationAllocation(reservationID2, start, alloc2, true);
+        createReservationAllocation(reservationID2, start, alloc2, true);
     Assert.assertNull(plan.getReservationById(reservationID2));
     try {
       plan.addReservation(rAllocation, false);
@@ -367,16 +493,18 @@ public class TestInMemoryPlan {
 
     for (int i = 0; i < alloc2.length; i++) {
       Assert.assertEquals(
-          Resource.newInstance(1024 * (alloc1[i] + alloc2[i] + i), alloc1[i]
-              + alloc2[i] + i), plan.getTotalCommittedResources(start + i));
+          Resource.newInstance(1024 * (alloc1[i] + alloc2[i] + i),
+              alloc1[i] + alloc2[i] + i),
+          plan.getTotalCommittedResources(start + i));
       Assert.assertEquals(
-          Resource.newInstance(1024 * (alloc1[i] + alloc2[i] + i), alloc1[i]
-              + alloc2[i] + i), userCons.getCapacityAtTime(start + i));
+          Resource.newInstance(1024 * (alloc1[i] + alloc2[i] + i),
+              alloc1[i] + alloc2[i] + i),
+          userCons.getCapacityAtTime(start + i));
     }
 
     // Now archive completed reservations
     when(clock.getTime()).thenReturn(106L);
-    when(policy.getValidWindow()).thenReturn(1L);
+    when(sharingPolicy.getValidWindow()).thenReturn(1L);
     try {
       // will only remove 2nd reservation as only that has fallen out of the
       // archival window
@@ -386,7 +514,7 @@ public class TestInMemoryPlan {
     }
     Assert.assertNotNull(plan.getReservationById(reservationID1));
     Assert.assertNull(plan.getReservationById(reservationID2));
-    checkAllocation(plan, alloc1, start);
+    checkAllocation(plan, alloc1, start, 0);
 
     when(clock.getTime()).thenReturn(107L);
     try {
@@ -411,15 +539,14 @@ public class TestInMemoryPlan {
 
   @Test
   public void testGetReservationsById() {
-    Plan plan =
-            new InMemoryPlan(queueMetrics, policy, agent, totalCapacity, 1L,
-                    resCalc, minAlloc, maxAlloc, planName, replanner, true, context);
+    Plan plan = new InMemoryPlan(queueMetrics, policy, agent, totalCapacity, 1L,
+        resCalc, minAlloc, maxAlloc, planName, replanner, true, context);
     ReservationId reservationID =
-            ReservationSystemTestUtil.getNewReservationId();
-    int[] alloc = {10, 10, 10, 10, 10, 10};
+        ReservationSystemTestUtil.getNewReservationId();
+    int[] alloc = { 10, 10, 10, 10, 10, 10 };
     int start = 100;
-    ReservationAllocation rAllocation = createReservationAllocation
-            (reservationID, start, alloc);
+    ReservationAllocation rAllocation =
+        createReservationAllocation(reservationID, start, alloc);
     Assert.assertNull(plan.getReservationById(reservationID));
     try {
       plan.addReservation(rAllocation, false);
@@ -429,31 +556,30 @@ public class TestInMemoryPlan {
 
     // Verify that get by reservation id works.
     Set<ReservationAllocation> rAllocations =
-            plan.getReservations(reservationID, null, "");
+        plan.getReservations(reservationID, null, "");
     Assert.assertTrue(rAllocations.size() == 1);
-    Assert.assertTrue(rAllocation.compareTo(
-            (ReservationAllocation) rAllocations.toArray()[0]) == 0);
+    Assert.assertTrue(rAllocation
+        .compareTo((ReservationAllocation) rAllocations.toArray()[0]) == 0);
 
     // Verify that get by reservation id works even when time range
     // and user is invalid.
     ReservationInterval interval = new ReservationInterval(0, 0);
     rAllocations = plan.getReservations(reservationID, interval, "invalid");
     Assert.assertTrue(rAllocations.size() == 1);
-    Assert.assertTrue(rAllocation.compareTo(
-            (ReservationAllocation) rAllocations.toArray()[0]) == 0);
+    Assert.assertTrue(rAllocation
+        .compareTo((ReservationAllocation) rAllocations.toArray()[0]) == 0);
   }
 
   @Test
   public void testGetReservationsByInvalidId() {
-    Plan plan =
-            new InMemoryPlan(queueMetrics, policy, agent, totalCapacity, 1L,
-                    resCalc, minAlloc, maxAlloc, planName, replanner, true, context);
+    Plan plan = new InMemoryPlan(queueMetrics, policy, agent, totalCapacity, 1L,
+        resCalc, minAlloc, maxAlloc, planName, replanner, true, context);
     ReservationId reservationID =
-            ReservationSystemTestUtil.getNewReservationId();
-    int[] alloc = {10, 10, 10, 10, 10, 10};
+        ReservationSystemTestUtil.getNewReservationId();
+    int[] alloc = { 10, 10, 10, 10, 10, 10 };
     int start = 100;
-    ReservationAllocation rAllocation = createReservationAllocation
-            (reservationID, start, alloc);
+    ReservationAllocation rAllocation =
+        createReservationAllocation(reservationID, start, alloc);
     Assert.assertNull(plan.getReservationById(reservationID));
     try {
       plan.addReservation(rAllocation, false);
@@ -463,23 +589,22 @@ public class TestInMemoryPlan {
 
     // If reservationId is null, then nothing is returned.
     ReservationId invalidReservationID =
-            ReservationSystemTestUtil.getNewReservationId();
+        ReservationSystemTestUtil.getNewReservationId();
     Set<ReservationAllocation> rAllocations =
-            plan.getReservations(invalidReservationID, null, "");
+        plan.getReservations(invalidReservationID, null, "");
     Assert.assertTrue(rAllocations.size() == 0);
   }
 
   @Test
   public void testGetReservationsByTimeInterval() {
-    Plan plan =
-            new InMemoryPlan(queueMetrics, policy, agent, totalCapacity, 1L,
-                    resCalc, minAlloc, maxAlloc, planName, replanner, true, context);
+    Plan plan = new InMemoryPlan(queueMetrics, policy, agent, totalCapacity, 1L,
+        resCalc, minAlloc, maxAlloc, planName, replanner, true, context);
     ReservationId reservationID =
-            ReservationSystemTestUtil.getNewReservationId();
-    int[] alloc = {10, 10, 10, 10, 10, 10};
+        ReservationSystemTestUtil.getNewReservationId();
+    int[] alloc = { 10, 10, 10, 10, 10, 10 };
     int start = 100;
-    ReservationAllocation rAllocation = createReservationAllocation
-            (reservationID, start, alloc);
+    ReservationAllocation rAllocation =
+        createReservationAllocation(reservationID, start, alloc);
     Assert.assertNull(plan.getReservationById(reservationID));
     try {
       plan.addReservation(rAllocation, false);
@@ -489,23 +614,24 @@ public class TestInMemoryPlan {
 
     // Verify that get by time interval works if the selection interval
     // completely overlaps with the allocation.
-    ReservationInterval interval = new ReservationInterval(rAllocation
-            .getStartTime(), rAllocation.getEndTime());
+    ReservationInterval interval = new ReservationInterval(
+        rAllocation.getStartTime(), rAllocation.getEndTime());
     Set<ReservationAllocation> rAllocations =
-            plan.getReservations(null, interval, "");
+        plan.getReservations(null, interval, "");
     Assert.assertTrue(rAllocations.size() == 1);
-    Assert.assertTrue(rAllocation.compareTo(
-            (ReservationAllocation) rAllocations.toArray()[0]) == 0);
+    Assert.assertTrue(rAllocation
+        .compareTo((ReservationAllocation) rAllocations.toArray()[0]) == 0);
 
     // Verify that get by time interval works if the selection interval
     // falls within the allocation
     long duration = rAllocation.getEndTime() - rAllocation.getStartTime();
-    interval = new ReservationInterval(rAllocation.getStartTime() + duration
-            * (long)0.3, rAllocation.getEndTime() - duration * (long)0.3);
+    interval = new ReservationInterval(
+        rAllocation.getStartTime() + duration * (long) 0.3,
+        rAllocation.getEndTime() - duration * (long) 0.3);
     rAllocations = plan.getReservations(null, interval, "");
     Assert.assertTrue(rAllocations.size() == 1);
-    Assert.assertTrue(rAllocation.compareTo(
-            (ReservationAllocation) rAllocations.toArray()[0]) == 0);
+    Assert.assertTrue(rAllocation
+        .compareTo((ReservationAllocation) rAllocations.toArray()[0]) == 0);
 
     // Verify that get by time interval selects 1 allocation if the end
     // time of the selection interval falls right at the start of the
@@ -513,13 +639,13 @@ public class TestInMemoryPlan {
     interval = new ReservationInterval(0, rAllocation.getStartTime());
     rAllocations = plan.getReservations(null, interval, "");
     Assert.assertTrue(rAllocations.size() == 1);
-    Assert.assertTrue(rAllocation.compareTo(
-            (ReservationAllocation) rAllocations.toArray()[0]) == 0);
+    Assert.assertTrue(rAllocation
+        .compareTo((ReservationAllocation) rAllocations.toArray()[0]) == 0);
 
     // Verify that get by time interval selects no reservations if the start
     // time of the selection interval falls right at the end of the allocation.
-    interval = new ReservationInterval(rAllocation
-            .getEndTime(), Long.MAX_VALUE);
+    interval =
+        new ReservationInterval(rAllocation.getEndTime(), Long.MAX_VALUE);
     rAllocations = plan.getReservations(null, interval, "");
     Assert.assertTrue(rAllocations.size() == 0);
 
@@ -532,15 +658,14 @@ public class TestInMemoryPlan {
 
   @Test
   public void testGetReservationsAtTime() {
-    Plan plan =
-            new InMemoryPlan(queueMetrics, policy, agent, totalCapacity, 1L,
-                    resCalc, minAlloc, maxAlloc, planName, replanner, true, context);
+    Plan plan = new InMemoryPlan(queueMetrics, policy, agent, totalCapacity, 1L,
+        resCalc, minAlloc, maxAlloc, planName, replanner, true, context);
     ReservationId reservationID =
-            ReservationSystemTestUtil.getNewReservationId();
-    int[] alloc = {10, 10, 10, 10, 10, 10};
+        ReservationSystemTestUtil.getNewReservationId();
+    int[] alloc = { 10, 10, 10, 10, 10, 10 };
     int start = 100;
-    ReservationAllocation rAllocation = createReservationAllocation
-            (reservationID, start, alloc);
+    ReservationAllocation rAllocation =
+        createReservationAllocation(reservationID, start, alloc);
     Assert.assertNull(plan.getReservationById(reservationID));
     try {
       plan.addReservation(rAllocation, false);
@@ -549,23 +674,22 @@ public class TestInMemoryPlan {
     }
 
     Set<ReservationAllocation> rAllocations =
-            plan.getReservationsAtTime(rAllocation.getStartTime());
+        plan.getReservationsAtTime(rAllocation.getStartTime());
     Assert.assertTrue(rAllocations.size() == 1);
-    Assert.assertTrue(rAllocation.compareTo(
-            (ReservationAllocation) rAllocations.toArray()[0]) == 0);
+    Assert.assertTrue(rAllocation
+        .compareTo((ReservationAllocation) rAllocations.toArray()[0]) == 0);
   }
 
   @Test
   public void testGetReservationsWithNoInput() {
-    Plan plan =
-            new InMemoryPlan(queueMetrics, policy, agent, totalCapacity, 1L,
-                    resCalc, minAlloc, maxAlloc, planName, replanner, true, context);
+    Plan plan = new InMemoryPlan(queueMetrics, policy, agent, totalCapacity, 1L,
+        resCalc, minAlloc, maxAlloc, planName, replanner, true, context);
     ReservationId reservationID =
-            ReservationSystemTestUtil.getNewReservationId();
-    int[] alloc = {10, 10, 10, 10, 10, 10};
+        ReservationSystemTestUtil.getNewReservationId();
+    int[] alloc = { 10, 10, 10, 10, 10, 10 };
     int start = 100;
-    ReservationAllocation rAllocation = createReservationAllocation
-            (reservationID, start, alloc);
+    ReservationAllocation rAllocation =
+        createReservationAllocation(reservationID, start, alloc);
     Assert.assertNull(plan.getReservationById(reservationID));
     try {
       plan.addReservation(rAllocation, false);
@@ -576,22 +700,21 @@ public class TestInMemoryPlan {
     // Verify that getReservations defaults to getting all reservations if no
     // reservationID, time interval, and user is provided,
     Set<ReservationAllocation> rAllocations =
-            plan.getReservations(null, null, "");
+        plan.getReservations(null, null, "");
     Assert.assertTrue(rAllocations.size() == 1);
-    Assert.assertTrue(rAllocation.compareTo(
-            (ReservationAllocation) rAllocations.toArray()[0]) == 0);
+    Assert.assertTrue(rAllocation
+        .compareTo((ReservationAllocation) rAllocations.toArray()[0]) == 0);
   }
 
   @Test
   public void testGetReservationsWithNoReservation() {
-    Plan plan =
-            new InMemoryPlan(queueMetrics, policy, agent, totalCapacity, 1L,
-                    resCalc, minAlloc, maxAlloc, planName, replanner, true, context);
+    Plan plan = new InMemoryPlan(queueMetrics, policy, agent, totalCapacity, 1L,
+        resCalc, minAlloc, maxAlloc, planName, replanner, true, context);
     // Verify that get reservation returns no entries if no queries are made.
 
     ReservationInterval interval = new ReservationInterval(0, Long.MAX_VALUE);
     Set<ReservationAllocation> rAllocations =
-            plan.getReservations(null, interval, "");
+        plan.getReservations(null, interval, "");
     Assert.assertTrue(rAllocations.size() == 0);
   }
 
@@ -600,7 +723,9 @@ public class TestInMemoryPlan {
     Assert.assertNotNull(plan.getReservationById(reservationID));
     Assert.assertEquals(rAllocation, plan.getReservationById(reservationID));
     Assert.assertTrue(((InMemoryPlan) plan).getAllReservations().size() == 1);
-    Assert.assertEquals(rAllocation.getEndTime(), plan.getLastEndTime());
+    if (rAllocation.getPeriodicity() <= 0) {
+      Assert.assertEquals(rAllocation.getEndTime(), plan.getLastEndTime());
+    }
     Assert.assertEquals(totalCapacity, plan.getTotalCapacity());
     Assert.assertEquals(minAlloc, plan.getMinimumAllocation());
     Assert.assertEquals(maxAlloc, plan.getMaximumAllocation());
@@ -610,7 +735,8 @@ public class TestInMemoryPlan {
   }
 
   private ReservationDefinition createSimpleReservationDefinition(long arrival,
-      long deadline, long duration, Collection<ReservationRequest> resources) {
+      long deadline, long duration, Collection<ReservationRequest> resources,
+      String recurrenceExpression) {
     // create a request with a single atomic ask
     ReservationDefinition rDef = new ReservationDefinitionPBImpl();
     ReservationRequests reqs = new ReservationRequestsPBImpl();
@@ -619,6 +745,7 @@ public class TestInMemoryPlan {
     rDef.setReservationRequests(reqs);
     rDef.setArrival(arrival);
     rDef.setDeadline(deadline);
+    rDef.setRecurrenceExpression(recurrenceExpression);
     return rDef;
   }
 
@@ -633,31 +760,43 @@ public class TestInMemoryPlan {
       } else {
         numContainers = alloc[i];
       }
-      ReservationRequest rr =
-          ReservationRequest.newInstance(Resource.newInstance(1024, 1),
-              (numContainers));
+      ReservationRequest rr = ReservationRequest
+          .newInstance(Resource.newInstance(1024, 1), (numContainers));
       req.put(new ReservationInterval(startTime + i, startTime + i + 1), rr);
     }
     return req;
   }
 
-  private ReservationAllocation createReservationAllocation(ReservationId
-            reservationID, int start, int[] alloc) {
-    return createReservationAllocation(reservationID, start, alloc, false);
+  private ReservationAllocation createReservationAllocation(
+      ReservationId reservationID, int start, int[] alloc) {
+    return createReservationAllocation(reservationID, start, alloc, false, "0");
+  }
+
+  private ReservationAllocation createReservationAllocation(
+      ReservationId reservationID, int start, int[] alloc, boolean isStep) {
+    return createReservationAllocation(reservationID, start, alloc, isStep,
+        "0");
+  }
+
+  private ReservationAllocation createReservationAllocation(
+      ReservationId reservationID, int start, int[] alloc,
+      String recurrenceExp) {
+    return createReservationAllocation(reservationID, start, alloc, false,
+        recurrenceExp);
   }
 
-  private ReservationAllocation createReservationAllocation(ReservationId
-            reservationID, int start, int[] alloc, boolean isStep) {
+  private ReservationAllocation createReservationAllocation(
+      ReservationId reservationID, int start, int[] alloc, boolean isStep,
+      String recurrenceExp) {
     Map<ReservationInterval, ReservationRequest> allocations =
-            generateAllocation(start, alloc, isStep);
+        generateAllocation(start, alloc, isStep);
     ReservationDefinition rDef =
-            createSimpleReservationDefinition(start, start + alloc.length,
-                    alloc.length, allocations.values());
+        createSimpleReservationDefinition(start, start + alloc.length,
+            alloc.length, allocations.values(), recurrenceExp);
     Map<ReservationInterval, Resource> allocs =
-            ReservationSystemUtil.toResources(allocations);
+        ReservationSystemUtil.toResources(allocations);
     return new InMemoryReservationAllocation(reservationID, rDef, user,
-            planName,
-                    start, start + alloc.length, allocs, resCalc, minAlloc);
+        planName, start, start + alloc.length, allocs, resCalc, minAlloc);
   }
 
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7996eca7/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestPeriodicRLESparseResourceAllocation.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/reservation/TestPeriodicRLESparseResourceAllocation.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestPeriodicRLESparseResourceAllocation.java
index 554eb58..457e2ee 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestPeriodicRLESparseResourceAllocation.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestPeriodicRLESparseResourceAllocation.java
@@ -19,26 +19,27 @@
 package org.apache.hadoop.yarn.server.resourcemanager.reservation;
 
 import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.server.resourcemanager.reservation.exceptions.PlanningException;
 import org.junit.Assert;
 import org.junit.Test;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 /**
- * Testing the class PeriodicRLESparseResourceAllocation.
+ * Testing the class {@link PeriodicRLESparseResourceAllocation}.
  */
+@SuppressWarnings("checkstyle:nowhitespaceafter")
 public class TestPeriodicRLESparseResourceAllocation {
 
-  private static final Logger LOG = LoggerFactory
-      .getLogger(TestPeriodicRLESparseResourceAllocation.class);
+  private static final Logger LOG =
+      LoggerFactory.getLogger(TestPeriodicRLESparseResourceAllocation.class);
 
   @Test
   public void testPeriodicCapacity() {
-    int[] alloc = {10, 7, 5, 2, 0};
-    long[] timeSteps = {0L, 5L, 10L, 15L, 19L};
-    RLESparseResourceAllocation rleSparseVector =
-        ReservationSystemTestUtil.generateRLESparseResourceAllocation(
-            alloc, timeSteps);
+    int[] alloc = { 10, 7, 5, 2, 0 };
+    long[] timeSteps = { 0L, 5L, 10L, 15L, 19L };
+    RLESparseResourceAllocation rleSparseVector = ReservationSystemTestUtil
+        .generateRLESparseResourceAllocation(alloc, timeSteps);
     PeriodicRLESparseResourceAllocation periodicVector =
         new PeriodicRLESparseResourceAllocation(rleSparseVector, 20L);
     LOG.info(periodicVector.toString());
@@ -54,43 +55,74 @@ public class TestPeriodicRLESparseResourceAllocation {
 
   @Test
   public void testMaxPeriodicCapacity() {
-    int[] alloc = {2, 5, 7, 10, 3, 4, 6, 8};
-    long[] timeSteps = {0L, 1L, 2L, 3L, 4L, 5L, 6L, 7L};
-    RLESparseResourceAllocation rleSparseVector =
-        ReservationSystemTestUtil.generateRLESparseResourceAllocation(
-            alloc, timeSteps);
+    int[] alloc = { 2, 5, 7, 10, 3, 4, 6, 8 };
+    long[] timeSteps = { 0L, 1L, 2L, 3L, 4L, 5L, 6L, 7L };
+    RLESparseResourceAllocation rleSparseVector = ReservationSystemTestUtil
+        .generateRLESparseResourceAllocation(alloc, timeSteps);
     PeriodicRLESparseResourceAllocation periodicVector =
         new PeriodicRLESparseResourceAllocation(rleSparseVector, 8L);
     LOG.info(periodicVector.toString());
-    Assert.assertEquals(
-        periodicVector.getMaximumPeriodicCapacity(0, 1),
+    Assert.assertEquals(periodicVector.getMaximumPeriodicCapacity(0, 1),
         Resource.newInstance(10, 10));
-    Assert.assertEquals(
-        periodicVector.getMaximumPeriodicCapacity(8, 2),
+    Assert.assertEquals(periodicVector.getMaximumPeriodicCapacity(8, 2),
         Resource.newInstance(7, 7));
-    Assert.assertEquals(
-        periodicVector.getMaximumPeriodicCapacity(16, 3),
+    Assert.assertEquals(periodicVector.getMaximumPeriodicCapacity(16, 3),
         Resource.newInstance(10, 10));
-    Assert.assertEquals(
-        periodicVector.getMaximumPeriodicCapacity(17, 4),
+    Assert.assertEquals(periodicVector.getMaximumPeriodicCapacity(17, 4),
         Resource.newInstance(5, 5));
-    Assert.assertEquals(
-        periodicVector.getMaximumPeriodicCapacity(32, 5),
+    Assert.assertEquals(periodicVector.getMaximumPeriodicCapacity(32, 5),
         Resource.newInstance(4, 4));
   }
 
   @Test
+  public void testMixPeriodicAndNonPeriodic() throws PlanningException {
+    int[] alloc = { 2, 5, 0 };
+    long[] timeSteps = { 1L, 2L, 3L };
+    RLESparseResourceAllocation tempPeriodic = ReservationSystemTestUtil
+        .generateRLESparseResourceAllocation(alloc, timeSteps);
+    PeriodicRLESparseResourceAllocation periodic =
+        new PeriodicRLESparseResourceAllocation(tempPeriodic, 10L);
+
+    int[] alloc2 = { 10, 10, 0 };
+    long[] timeSteps2 = { 12L, 13L, 14L };
+    RLESparseResourceAllocation nonPeriodic = ReservationSystemTestUtil
+        .generateRLESparseResourceAllocation(alloc2, timeSteps2);
+
+    RLESparseResourceAllocation merged =
+        RLESparseResourceAllocation.merge(nonPeriodic.getResourceCalculator(),
+            Resource.newInstance(100 * 1024, 100), periodic, nonPeriodic,
+            RLESparseResourceAllocation.RLEOperator.add, 2, 25);
+
+    Assert.assertEquals(Resource.newInstance(5, 5),
+        merged.getCapacityAtTime(2L));
+    Assert.assertEquals(Resource.newInstance(0, 0),
+        merged.getCapacityAtTime(3L));
+    Assert.assertEquals(Resource.newInstance(2, 2),
+        merged.getCapacityAtTime(11L));
+    Assert.assertEquals(Resource.newInstance(15, 15),
+        merged.getCapacityAtTime(12L));
+    Assert.assertEquals(Resource.newInstance(10, 10),
+        merged.getCapacityAtTime(13L));
+    Assert.assertEquals(Resource.newInstance(0, 0),
+        merged.getCapacityAtTime(14L));
+    Assert.assertEquals(Resource.newInstance(2, 2),
+        merged.getCapacityAtTime(21L));
+    Assert.assertEquals(Resource.newInstance(5, 5),
+        merged.getCapacityAtTime(22L));
+    Assert.assertEquals(Resource.newInstance(0, 0),
+        merged.getCapacityAtTime(23L));
+  }
+
+  @Test
   public void testSetCapacityInInterval() {
-    int[] alloc = {2, 5, 0};
-    long[] timeSteps = {1L, 2L, 3L};
-    RLESparseResourceAllocation rleSparseVector =
-        ReservationSystemTestUtil.generateRLESparseResourceAllocation(
-            alloc, timeSteps);
+    int[] alloc = { 2, 5, 0 };
+    long[] timeSteps = { 1L, 2L, 3L };
+    RLESparseResourceAllocation rleSparseVector = ReservationSystemTestUtil
+        .generateRLESparseResourceAllocation(alloc, timeSteps);
     PeriodicRLESparseResourceAllocation periodicVector =
         new PeriodicRLESparseResourceAllocation(rleSparseVector, 10L);
     ReservationInterval interval = new ReservationInterval(5L, 10L);
-    periodicVector.addInterval(
-        interval, Resource.newInstance(8, 8));
+    periodicVector.addInterval(interval, Resource.newInstance(8, 8));
     Assert.assertEquals(Resource.newInstance(8, 8),
         periodicVector.getCapacityAtTime(5L));
     Assert.assertEquals(Resource.newInstance(8, 8),
@@ -99,21 +131,20 @@ public class TestPeriodicRLESparseResourceAllocation {
         periodicVector.getCapacityAtTime(10L));
     Assert.assertEquals(Resource.newInstance(0, 0),
         periodicVector.getCapacityAtTime(0L));
-    Assert.assertFalse(periodicVector.addInterval(
-        new ReservationInterval(7L, 12L), Resource.newInstance(8, 8)));
+    // Assert.assertFalse(periodicVector.addInterval(
+    // new ReservationInterval(7L, 12L), Resource.newInstance(8, 8)));
   }
 
   public void testRemoveInterval() {
-    int[] alloc = {2, 5, 3, 4, 0};
-    long[] timeSteps = {1L, 3L, 5L, 7L, 9L};
-    RLESparseResourceAllocation rleSparseVector =
-        ReservationSystemTestUtil.generateRLESparseResourceAllocation(
-            alloc, timeSteps);
+    int[] alloc = { 2, 5, 3, 4, 0 };
+    long[] timeSteps = { 1L, 3L, 5L, 7L, 9L };
+    RLESparseResourceAllocation rleSparseVector = ReservationSystemTestUtil
+        .generateRLESparseResourceAllocation(alloc, timeSteps);
     PeriodicRLESparseResourceAllocation periodicVector =
         new PeriodicRLESparseResourceAllocation(rleSparseVector, 10L);
     ReservationInterval interval = new ReservationInterval(3L, 7L);
-    Assert.assertTrue(periodicVector.removeInterval(
-        interval, Resource.newInstance(3, 3)));
+    Assert.assertTrue(
+        periodicVector.removeInterval(interval, Resource.newInstance(3, 3)));
     Assert.assertEquals(Resource.newInstance(2, 2),
         periodicVector.getCapacityAtTime(1L));
     Assert.assertEquals(Resource.newInstance(2, 2),

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7996eca7/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestRLESparseResourceAllocation.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/reservation/TestRLESparseResourceAllocation.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestRLESparseResourceAllocation.java
index bfe46e1..0027ceb 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestRLESparseResourceAllocation.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestRLESparseResourceAllocation.java
@@ -40,10 +40,14 @@ import org.junit.Test;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+/**
+ * Testing the class {@link RLESparseResourceAllocation}.
+ */
+@SuppressWarnings("checkstyle:nowhitespaceafter")
 public class TestRLESparseResourceAllocation {
 
-  private static final Logger LOG = LoggerFactory
-      .getLogger(TestRLESparseResourceAllocation.class);
+  private static final Logger LOG =
+      LoggerFactory.getLogger(TestRLESparseResourceAllocation.class);
 
   @Test
   public void testMergeAdd() throws PlanningException {
@@ -196,7 +200,8 @@ public class TestRLESparseResourceAllocation {
       // Expected!
     }
 
-    // Testing that the subtractTestNonNegative detects problems even if only one
+    // Testing that the subtractTestNonNegative detects problems even if only
+    // one
     // of the resource dimensions is "<0"
     a.put(10L, Resource.newInstance(10, 5));
     b.put(11L, Resource.newInstance(5, 6));
@@ -286,9 +291,8 @@ public class TestRLESparseResourceAllocation {
   public void testRangeOverlapping() {
     ResourceCalculator resCalc = new DefaultResourceCalculator();
 
-    RLESparseResourceAllocation r =
-        new RLESparseResourceAllocation(resCalc);
-    int[] alloc = {10, 10, 10, 10, 10, 10};
+    RLESparseResourceAllocation r = new RLESparseResourceAllocation(resCalc);
+    int[] alloc = { 10, 10, 10, 10, 10, 10 };
     int start = 100;
     Set<Entry<ReservationInterval, Resource>> inputs =
         generateAllocation(start, alloc, false).entrySet();
@@ -299,9 +303,9 @@ public class TestRLESparseResourceAllocation {
     long d = r.getLatestNonNullTime();
 
     // tries to trigger "out-of-range" bug
-    r =  r.getRangeOverlapping(s, d);
-    r = r.getRangeOverlapping(s-1, d-1);
-    r = r.getRangeOverlapping(s+1, d+1);
+    r = r.getRangeOverlapping(s, d);
+    r = r.getRangeOverlapping(s - 1, d - 1);
+    r = r.getRangeOverlapping(s + 1, d + 1);
   }
 
   @Test
@@ -370,25 +374,29 @@ public class TestRLESparseResourceAllocation {
     // Current bug prevents this to pass. The RLESparseResourceAllocation
     // does not handle removal of "partial"
     // allocations correctly.
-    Assert.assertEquals(102400, rleSparseVector.getCapacityAtTime(10)
-        .getMemorySize());
-    Assert.assertEquals(0, rleSparseVector.getCapacityAtTime(13).getMemorySize());
-    Assert.assertEquals(0, rleSparseVector.getCapacityAtTime(19).getMemorySize());
-    Assert.assertEquals(102400, rleSparseVector.getCapacityAtTime(21)
-        .getMemorySize());
-    Assert.assertEquals(2 * 102400, rleSparseVector.getCapacityAtTime(26)
-        .getMemorySize());
+    Assert.assertEquals(102400,
+        rleSparseVector.getCapacityAtTime(10).getMemorySize());
+    Assert.assertEquals(0,
+        rleSparseVector.getCapacityAtTime(13).getMemorySize());
+    Assert.assertEquals(0,
+        rleSparseVector.getCapacityAtTime(19).getMemorySize());
+    Assert.assertEquals(102400,
+        rleSparseVector.getCapacityAtTime(21).getMemorySize());
+    Assert.assertEquals(2 * 102400,
+        rleSparseVector.getCapacityAtTime(26).getMemorySize());
 
     ReservationInterval riRemove2 = new ReservationInterval(9, 13);
     rleSparseVector.removeInterval(riRemove2, rr);
     LOG.info(rleSparseVector.toString());
 
-    Assert.assertEquals(0, rleSparseVector.getCapacityAtTime(11).getMemorySize());
-    Assert.assertEquals(-102400, rleSparseVector.getCapacityAtTime(9)
-        .getMemorySize());
-    Assert.assertEquals(0, rleSparseVector.getCapacityAtTime(13).getMemorySize());
-    Assert.assertEquals(102400, rleSparseVector.getCapacityAtTime(20)
-        .getMemorySize());
+    Assert.assertEquals(0,
+        rleSparseVector.getCapacityAtTime(11).getMemorySize());
+    Assert.assertEquals(-102400,
+        rleSparseVector.getCapacityAtTime(9).getMemorySize());
+    Assert.assertEquals(0,
+        rleSparseVector.getCapacityAtTime(13).getMemorySize());
+    Assert.assertEquals(102400,
+        rleSparseVector.getCapacityAtTime(20).getMemorySize());
 
   }
 
@@ -500,7 +508,8 @@ public class TestRLESparseResourceAllocation {
     }
     mapAllocations = rleSparseVector.toIntervalMap();
     Assert.assertTrue(mapAllocations.size() == 5);
-    for (Entry<ReservationInterval, Resource> entry : mapAllocations.entrySet()) {
+    for (Entry<ReservationInterval, Resource> entry : mapAllocations
+        .entrySet()) {
       ReservationInterval interval = entry.getKey();
       Resource resource = entry.getValue();
       if (interval.getStartTime() == 101L) {
@@ -526,59 +535,46 @@ public class TestRLESparseResourceAllocation {
 
   @Test
   public void testMaxPeriodicCapacity() {
-    long[] timeSteps = {0L, 1L, 2L, 3L, 4L, 5L, 6L, 7L};
-    int[] alloc = {2, 5, 7, 10, 3, 4, 6, 8};
-    RLESparseResourceAllocation rleSparseVector =
-        ReservationSystemTestUtil.generateRLESparseResourceAllocation(
-            alloc, timeSteps);
+    long[] timeSteps = { 0L, 1L, 2L, 3L, 4L, 5L, 6L, 7L };
+    int[] alloc = { 2, 5, 7, 10, 3, 4, 6, 8 };
+    RLESparseResourceAllocation rleSparseVector = ReservationSystemTestUtil
+        .generateRLESparseResourceAllocation(alloc, timeSteps);
     LOG.info(rleSparseVector.toString());
-    Assert.assertEquals(
-        rleSparseVector.getMaximumPeriodicCapacity(0, 1),
+    Assert.assertEquals(rleSparseVector.getMaximumPeriodicCapacity(0, 1),
         Resource.newInstance(10, 10));
-    Assert.assertEquals(
-        rleSparseVector.getMaximumPeriodicCapacity(0, 2),
+    Assert.assertEquals(rleSparseVector.getMaximumPeriodicCapacity(0, 2),
         Resource.newInstance(7, 7));
-    Assert.assertEquals(
-        rleSparseVector.getMaximumPeriodicCapacity(0, 3),
+    Assert.assertEquals(rleSparseVector.getMaximumPeriodicCapacity(0, 3),
         Resource.newInstance(10, 10));
-    Assert.assertEquals(
-        rleSparseVector.getMaximumPeriodicCapacity(0, 4),
+    Assert.assertEquals(rleSparseVector.getMaximumPeriodicCapacity(0, 4),
         Resource.newInstance(3, 3));
-    Assert.assertEquals(
-        rleSparseVector.getMaximumPeriodicCapacity(0, 5),
+    Assert.assertEquals(rleSparseVector.getMaximumPeriodicCapacity(0, 5),
         Resource.newInstance(4, 4));
-    Assert.assertEquals(
-        rleSparseVector.getMaximumPeriodicCapacity(0, 5),
+    Assert.assertEquals(rleSparseVector.getMaximumPeriodicCapacity(0, 5),
         Resource.newInstance(4, 4));
-    Assert.assertEquals(
-        rleSparseVector.getMaximumPeriodicCapacity(7, 5),
+    Assert.assertEquals(rleSparseVector.getMaximumPeriodicCapacity(7, 5),
         Resource.newInstance(8, 8));
-    Assert.assertEquals(
-        rleSparseVector.getMaximumPeriodicCapacity(10, 3),
+    Assert.assertEquals(rleSparseVector.getMaximumPeriodicCapacity(10, 3),
         Resource.newInstance(0, 0));
   }
 
   @Test
   public void testGetMinimumCapacityInInterval() {
-    long[] timeSteps = {0L, 1L, 2L, 3L, 4L, 5L, 6L, 7L};
-    int[] alloc = {2, 5, 7, 10, 3, 4, 0, 8};
-    RLESparseResourceAllocation rleSparseVector =
-        ReservationSystemTestUtil.generateRLESparseResourceAllocation(
-            alloc, timeSteps);
+    long[] timeSteps = { 0L, 1L, 2L, 3L, 4L, 5L, 6L, 7L };
+    int[] alloc = { 2, 5, 7, 10, 3, 4, 0, 8 };
+    RLESparseResourceAllocation rleSparseVector = ReservationSystemTestUtil
+        .generateRLESparseResourceAllocation(alloc, timeSteps);
     LOG.info(rleSparseVector.toString());
-    Assert.assertEquals(
-        rleSparseVector.getMinimumCapacityInInterval(
-            new ReservationInterval(1L, 3L)), Resource.newInstance(5, 5));
-    Assert.assertEquals(
-        rleSparseVector.getMinimumCapacityInInterval(
-            new ReservationInterval(2L, 5L)), Resource.newInstance(3, 3));
-    Assert.assertEquals(
-        rleSparseVector.getMinimumCapacityInInterval(
-            new ReservationInterval(1L, 7L)), Resource.newInstance(0, 0));
+    Assert.assertEquals(rleSparseVector.getMinimumCapacityInInterval(
+        new ReservationInterval(1L, 3L)), Resource.newInstance(5, 5));
+    Assert.assertEquals(rleSparseVector.getMinimumCapacityInInterval(
+        new ReservationInterval(2L, 5L)), Resource.newInstance(3, 3));
+    Assert.assertEquals(rleSparseVector.getMinimumCapacityInInterval(
+        new ReservationInterval(1L, 7L)), Resource.newInstance(0, 0));
   }
 
-  private void setupArrays(
-      TreeMap<Long, Resource> a, TreeMap<Long, Resource> b) {
+  private void setupArrays(TreeMap<Long, Resource> a,
+      TreeMap<Long, Resource> b) {
     a.put(10L, Resource.newInstance(5, 5));
     a.put(20L, Resource.newInstance(10, 10));
     a.put(30L, Resource.newInstance(15, 15));
@@ -620,8 +616,8 @@ public class TestRLESparseResourceAllocation {
         numContainers = alloc[i];
       }
       req.put(new ReservationInterval(startTime + i, startTime + i + 1),
-          ReservationSystemUtil.toResource(ReservationRequest.newInstance(
-              Resource.newInstance(1024, 1), (numContainers))));
+          ReservationSystemUtil.toResource(ReservationRequest
+              .newInstance(Resource.newInstance(1024, 1), (numContainers))));
     }
     return req;
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7996eca7/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/planning/TestSimpleCapacityReplanner.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/reservation/planning/TestSimpleCapacityReplanner.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/planning/TestSimpleCapacityReplanner.java
index c4f94c2..ddd290d 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/planning/TestSimpleCapacityReplanner.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/planning/TestSimpleCapacityReplanner.java
@@ -31,6 +31,7 @@ import org.apache.hadoop.yarn.api.records.ReservationDefinition;
 import org.apache.hadoop.yarn.api.records.ReservationId;
 import org.apache.hadoop.yarn.api.records.ReservationRequest;
 import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
 import org.apache.hadoop.yarn.server.resourcemanager.reservation.InMemoryPlan;
 import org.apache.hadoop.yarn.server.resourcemanager.reservation.InMemoryReservationAllocation;
@@ -78,9 +79,10 @@ public class TestSimpleCapacityReplanner {
     enf.init("blah", conf);
 
     // Initialize the plan with more resources
-    InMemoryPlan plan =
-        new InMemoryPlan(queueMetrics, policy, agent, clusterCapacity, step,
-            res, minAlloc, maxAlloc, "dedicated", enf, true, context, clock);
+    InMemoryPlan plan = new InMemoryPlan(queueMetrics, policy, agent,
+        clusterCapacity, step, res, minAlloc, maxAlloc, "dedicated", enf, true,
+        YarnConfiguration.DEFAULT_RM_RESERVATION_SYSTEM_MAX_PERIODICITY,
+        context, clock);
 
     // add reservation filling the plan (separating them 1ms, so we are sure
     // s2 follows s1 on acceptance


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[11/37] hadoop git commit: HDFS-11882. Precisely calculate acked length of striped block groups in updatePipeline.

Posted by ae...@apache.org.
HDFS-11882. Precisely calculate acked length of striped block groups in updatePipeline.


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

Branch: refs/heads/HDFS-7240
Commit: ccd2ac60ecc5fccce56debf21a068e663c1d5f11
Parents: 0ba8ff4
Author: Andrew Wang <wa...@apache.org>
Authored: Tue Sep 5 14:16:03 2017 -0700
Committer: Andrew Wang <wa...@apache.org>
Committed: Tue Sep 5 14:16:03 2017 -0700

----------------------------------------------------------------------
 .../hadoop/hdfs/DFSStripedOutputStream.java     | 160 +++++++++++++++++--
 .../TestDFSStripedOutputStreamWithFailure.java  |  77 ++++++++-
 2 files changed, 222 insertions(+), 15 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/ccd2ac60/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSStripedOutputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSStripedOutputStream.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSStripedOutputStream.java
index d5206d1..408b325 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSStripedOutputStream.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSStripedOutputStream.java
@@ -641,7 +641,7 @@ public class DFSStripedOutputStream extends DFSOutputStream
       // wait till all the healthy streamers to
       // 1) get the updated block info
       // 2) create new block outputstream
-      newFailed = waitCreatingNewStreams(healthySet);
+      newFailed = waitCreatingStreamers(healthySet);
       if (newFailed.size() + failedStreamers.size() >
           numAllBlocks - numDataBlocks) {
         throw new IOException(
@@ -668,6 +668,14 @@ public class DFSStripedOutputStream extends DFSOutputStream
     }
   }
 
+  /**
+   * Check if the streamers were successfully updated, adding failed streamers
+   * in the <i>failed</i> return parameter.
+   * @param failed Return parameter containing failed streamers from
+   *               <i>streamers</i>.
+   * @param streamers Set of streamers that are being updated
+   * @return total number of successful updates and failures
+   */
   private int checkStreamerUpdates(Set<StripedDataStreamer> failed,
       Set<StripedDataStreamer> streamers) {
     for (StripedDataStreamer streamer : streamers) {
@@ -682,7 +690,15 @@ public class DFSStripedOutputStream extends DFSOutputStream
     return coordinator.updateStreamerMap.size() + failed.size();
   }
 
-  private Set<StripedDataStreamer> waitCreatingNewStreams(
+  /**
+   * Waits for streamers to be created.
+   *
+   * @param healthyStreamers Set of healthy streamers
+   * @return Set of streamers that failed.
+   *
+   * @throws IOException
+   */
+  private Set<StripedDataStreamer> waitCreatingStreamers(
       Set<StripedDataStreamer> healthyStreamers) throws IOException {
     Set<StripedDataStreamer> failed = new HashSet<>();
     final int expectedNum = healthyStreamers.size();
@@ -773,9 +789,10 @@ public class DFSStripedOutputStream extends DFSOutputStream
       }
     }
 
-    // should update the block group length based on the acked length
+    // Update the NameNode with the acked length of the block group
+    // Save and restore the unacked length
     final long sentBytes = currentBlockGroup.getNumBytes();
-    final long ackedBytes = getNumAckedStripes() * cellSize * numDataBlocks;
+    final long ackedBytes = getAckedLength();
     Preconditions.checkState(ackedBytes <= sentBytes,
         "Acked:" + ackedBytes + ", Sent:" + sentBytes);
     currentBlockGroup.setNumBytes(ackedBytes);
@@ -787,23 +804,140 @@ public class DFSStripedOutputStream extends DFSOutputStream
   }
 
   /**
-   * Get the number of acked stripes. An acked stripe means at least data block
-   * number size cells of the stripe were acked.
+   * Return the length of each block in the block group.
+   * Unhealthy blocks have a length of -1.
+   *
+   * @return List of block lengths.
    */
-  private long getNumAckedStripes() {
-    int minStripeNum = Integer.MAX_VALUE;
+  private List<Long> getBlockLengths() {
+    List<Long> blockLengths = new ArrayList<>(numAllBlocks);
     for (int i = 0; i < numAllBlocks; i++) {
       final StripedDataStreamer streamer = getStripedDataStreamer(i);
+      long numBytes = -1;
       if (streamer.isHealthy()) {
-        int curStripeNum = 0;
         if (streamer.getBlock() != null) {
-          curStripeNum = (int) (streamer.getBlock().getNumBytes() / cellSize);
+          numBytes = streamer.getBlock().getNumBytes();
         }
-        minStripeNum = Math.min(curStripeNum, minStripeNum);
+      }
+      blockLengths.add(numBytes);
+    }
+    return blockLengths;
+  }
+
+  /**
+   * Get the length of acked bytes in the block group.
+   *
+   * <p>
+   *   A full stripe is acked when at least numDataBlocks streamers have
+   *   the corresponding cells of the stripe, and all previous full stripes are
+   *   also acked. This enforces the constraint that there is at most one
+   *   partial stripe.
+   * </p>
+   * <p>
+   *   Partial stripes write all parity cells. Empty data cells are not written.
+   *   Parity cells are the length of the longest data cell(s). For example,
+   *   with RS(3,2), if we have data cells with lengths [1MB, 64KB, 0], the
+   *   parity blocks will be length [1MB, 1MB].
+   * </p>
+   * <p>
+   *   To be considered acked, a partial stripe needs at least numDataBlocks
+   *   empty or written cells.
+   * </p>
+   * <p>
+   *   Currently, partial stripes can only happen when closing the file at a
+   *   non-stripe boundary, but this could also happen during (currently
+   *   unimplemented) hflush/hsync support.
+   * </p>
+   */
+  private long getAckedLength() {
+    // Determine the number of full stripes that are sufficiently durable
+    final long sentBytes = currentBlockGroup.getNumBytes();
+    final long numFullStripes = sentBytes / numDataBlocks / cellSize;
+    final long fullStripeLength = numFullStripes * numDataBlocks * cellSize;
+    assert fullStripeLength <= sentBytes : "Full stripe length can't be " +
+        "greater than the block group length";
+
+    long ackedLength = 0;
+
+    // Determine the length contained by at least `numDataBlocks` blocks.
+    // Since it's sorted, all the blocks after `offset` are at least as long,
+    // and there are at least `numDataBlocks` at or after `offset`.
+    List<Long> blockLengths = Collections.unmodifiableList(getBlockLengths());
+    List<Long> sortedBlockLengths = new ArrayList<>(blockLengths);
+    Collections.sort(sortedBlockLengths);
+    if (numFullStripes > 0) {
+      final int offset = sortedBlockLengths.size() - numDataBlocks;
+      ackedLength = sortedBlockLengths.get(offset) * numDataBlocks;
+    }
+
+    // If the acked length is less than the expected full stripe length, then
+    // we're missing a full stripe. Return the acked length.
+    if (ackedLength < fullStripeLength) {
+      return ackedLength;
+    }
+    // If the expected length is exactly a stripe boundary, then we're also done
+    if (ackedLength == sentBytes) {
+      return ackedLength;
+    }
+
+    /*
+    Otherwise, we're potentially dealing with a partial stripe.
+    The partial stripe is laid out as follows:
+
+      0 or more full data cells, `cellSize` in length.
+      0 or 1 partial data cells.
+      0 or more empty data cells.
+      `numParityBlocks` parity cells, the length of the longest data cell.
+
+    If the partial stripe is sufficiently acked, we'll update the ackedLength.
+    */
+
+    // How many full and empty data cells do we expect?
+    final int numFullDataCells = (int)
+        ((sentBytes - fullStripeLength) / cellSize);
+    final int partialLength = (int) (sentBytes - fullStripeLength) % cellSize;
+    final int numPartialDataCells = partialLength == 0 ? 0 : 1;
+    final int numEmptyDataCells = numDataBlocks - numFullDataCells -
+        numPartialDataCells;
+    // Calculate the expected length of the parity blocks.
+    final int parityLength = numFullDataCells > 0 ? cellSize : partialLength;
+
+    final long fullStripeBlockOffset = fullStripeLength / numDataBlocks;
+
+    // Iterate through each type of streamers, checking the expected length.
+    long[] expectedBlockLengths = new long[numAllBlocks];
+    int idx = 0;
+    // Full cells
+    for (; idx < numFullDataCells; idx++) {
+      expectedBlockLengths[idx] = fullStripeBlockOffset + cellSize;
+    }
+    // Partial cell
+    for (; idx < numFullDataCells + numPartialDataCells; idx++) {
+      expectedBlockLengths[idx] = fullStripeBlockOffset + partialLength;
+    }
+    // Empty cells
+    for (; idx < numFullDataCells + numPartialDataCells + numEmptyDataCells;
+         idx++) {
+      expectedBlockLengths[idx] = fullStripeBlockOffset;
+    }
+    // Parity cells
+    for (; idx < numAllBlocks; idx++) {
+      expectedBlockLengths[idx] = fullStripeBlockOffset + parityLength;
+    }
+
+    // Check expected lengths against actual streamer lengths.
+    // Update if we have sufficient durability.
+    int numBlocksWithCorrectLength = 0;
+    for (int i = 0; i < numAllBlocks; i++) {
+      if (blockLengths.get(i) == expectedBlockLengths[i]) {
+        numBlocksWithCorrectLength++;
       }
     }
-    assert minStripeNum != Integer.MAX_VALUE;
-    return minStripeNum;
+    if (numBlocksWithCorrectLength >= numDataBlocks) {
+      ackedLength = sentBytes;
+    }
+
+    return ackedLength;
   }
 
   private int stripeDataSize() {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ccd2ac60/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStreamWithFailure.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStreamWithFailure.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStreamWithFailure.java
index 9915a2f..f63a353 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStreamWithFailure.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStreamWithFailure.java
@@ -45,7 +45,6 @@ import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.log4j.Level;
 import org.junit.Assert;
-import org.junit.Assume;
 import org.junit.Before;
 import org.junit.Test;
 
@@ -61,6 +60,7 @@ import java.util.concurrent.atomic.AtomicInteger;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
+import static org.junit.Assume.assumeTrue;
 
 /**
  * Test striped file write operation with data node failures.
@@ -391,6 +391,79 @@ public class TestDFSStripedOutputStreamWithFailure {
   }
 
   /**
+   * When the two DataNodes with partial data blocks fail.
+   */
+  @Test
+  public void runTestWithDifferentLengths() throws Exception {
+    assumeTrue("Skip this test case in the subclasses. Once is enough.",
+        this.getClass().equals(TestDFSStripedOutputStreamWithFailure.class));
+
+    final HdfsConfiguration conf = newHdfsConfiguration();
+
+    final int[] fileLengths = {
+        // Full stripe then partial on cell boundary
+        cellSize * (dataBlocks * 2 - 2),
+        // Full stripe and a partial on non-cell boundary
+        (cellSize * dataBlocks) + 123,
+    };
+    try {
+      for (int length: fileLengths) {
+        // select the two DNs with partial block to kill
+        final int[] dnIndex = {dataBlocks - 2, dataBlocks - 1};
+        final int[] killPos = getKillPositions(length, dnIndex.length);
+        try {
+          LOG.info("runTestWithMultipleFailure2: length==" + length
+              + ", killPos=" + Arrays.toString(killPos)
+              + ", dnIndex=" + Arrays.toString(dnIndex));
+          setup(conf);
+          runTest(length, killPos, dnIndex, false);
+        } catch (Throwable e) {
+          final String err = "failed, killPos=" + Arrays.toString(killPos)
+              + ", dnIndex=" + Arrays.toString(dnIndex) + ", length=" + length;
+          LOG.error(err);
+          throw e;
+        }
+      }
+    } finally {
+      tearDown();
+    }
+  }
+
+  /**
+   * Test writing very short EC files with many failures.
+   */
+  @Test
+  public void runTestWithShortStripe() throws Exception {
+    assumeTrue("Skip this test case in the subclasses. Once is enough.",
+        this.getClass().equals(TestDFSStripedOutputStreamWithFailure.class));
+
+    final HdfsConfiguration conf = newHdfsConfiguration();
+    // Write a file with a 1 cell partial stripe
+    final int length = cellSize - 123;
+    // Kill all but one DN
+    final int[] dnIndex = new int[dataBlocks + parityBlocks - 1];
+    for (int i = 0; i < dnIndex.length; i++) {
+      dnIndex[i] = i;
+    }
+    final int[] killPos = getKillPositions(length, dnIndex.length);
+
+    try {
+      LOG.info("runTestWithShortStripe: length==" + length + ", killPos="
+          + Arrays.toString(killPos) + ", dnIndex="
+          + Arrays.toString(dnIndex));
+      setup(conf);
+      runTest(length, killPos, dnIndex, false);
+    } catch (Throwable e) {
+      final String err = "failed, killPos=" + Arrays.toString(killPos)
+          + ", dnIndex=" + Arrays.toString(dnIndex) + ", length=" + length;
+      LOG.error(err);
+      throw e;
+    } finally {
+      tearDown();
+    }
+  }
+
+  /**
    * runTest implementation.
    * @param length file length
    * @param killPos killing positions in ascending order
@@ -558,7 +631,7 @@ public class TestDFSStripedOutputStreamWithFailure {
 
   private void run(int offset) {
     int base = getBase();
-    Assume.assumeTrue(base >= 0);
+    assumeTrue(base >= 0);
     final int i = offset + base;
     final Integer length = getLength(i);
     if (length == null) {


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[35/37] hadoop git commit: HADOOP-14774. S3A case "testRandomReadOverBuffer" failed due to improper range parameter. Contributed by Yonger.

Posted by ae...@apache.org.
HADOOP-14774. S3A case "testRandomReadOverBuffer" failed due to improper range parameter.
Contributed by Yonger.


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

Branch: refs/heads/HDFS-7240
Commit: 83449abb40633a36b22de538cf82fca240af067e
Parents: c41118a
Author: Steve Loughran <st...@apache.org>
Authored: Thu Sep 7 19:55:56 2017 +0100
Committer: Steve Loughran <st...@apache.org>
Committed: Thu Sep 7 19:55:56 2017 +0100

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


http://git-wip-us.apache.org/repos/asf/hadoop/blob/83449abb/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInputStream.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInputStream.java
index b88b7c1..94d7701 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInputStream.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInputStream.java
@@ -152,7 +152,7 @@ public class S3AInputStream extends FSInputStream implements CanSetReadahead {
     streamStatistics.streamOpened();
     try {
       GetObjectRequest request = new GetObjectRequest(bucket, key)
-          .withRange(targetPos, contentRangeFinish);
+          .withRange(targetPos, contentRangeFinish - 1);
       if (S3AEncryptionMethods.SSE_C.equals(serverSideEncryptionAlgorithm) &&
           StringUtils.isNotBlank(serverSideEncryptionKey)){
         request.setSSECustomerKey(new SSECustomerKey(serverSideEncryptionKey));


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[33/37] hadoop git commit: HADOOP-14520. WASB: Block compaction for Azure Block Blobs. Contributed by Georgi Chalakov

Posted by ae...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/13eda500/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestNativeAzureFileSystemBlockCompaction.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestNativeAzureFileSystemBlockCompaction.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestNativeAzureFileSystemBlockCompaction.java
new file mode 100644
index 0000000..820ce4f
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestNativeAzureFileSystemBlockCompaction.java
@@ -0,0 +1,266 @@
+/*
+ * 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.fs.azure;
+
+import com.microsoft.azure.storage.blob.BlockEntry;
+import org.apache.commons.lang.RandomStringUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.contract.ContractTestUtils;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.io.ByteArrayOutputStream;
+import java.io.OutputStream;
+import java.net.URI;
+import java.util.List;
+
+/**
+ * Test class that runs WASB block compaction process for block blobs.
+ */
+
+public class TestNativeAzureFileSystemBlockCompaction extends AbstractWasbTestBase {
+
+  private static final String TEST_FILE = "/user/active/test.dat";
+  private static final Path TEST_PATH = new Path(TEST_FILE);
+
+  private static final String TEST_FILE_NORMAL = "/user/normal/test.dat";
+  private static final Path TEST_PATH_NORMAL = new Path(TEST_FILE_NORMAL);
+
+  private AzureBlobStorageTestAccount testAccount = null;
+
+  @Before
+  public void setUp() throws Exception {
+    super.setUp();
+    testAccount = createTestAccount();
+    fs = testAccount.getFileSystem();
+    Configuration conf = fs.getConf();
+    conf.setBoolean(NativeAzureFileSystem.APPEND_SUPPORT_ENABLE_PROPERTY_NAME, true);
+    conf.set(AzureNativeFileSystemStore.KEY_BLOCK_BLOB_WITH_COMPACTION_DIRECTORIES, "/user/active");
+    URI uri = fs.getUri();
+    fs.initialize(uri, conf);
+  }
+
+  /*
+   * Helper method that creates test data of size provided by the
+   * "size" parameter.
+   */
+  private static byte[] getTestData(int size) {
+    byte[] testData = new byte[size];
+    System.arraycopy(RandomStringUtils.randomAlphabetic(size).getBytes(), 0, testData, 0, size);
+    return testData;
+  }
+
+  @Override
+  protected AzureBlobStorageTestAccount createTestAccount() throws Exception {
+    return AzureBlobStorageTestAccount.create();
+  }
+
+  private BlockBlobAppendStream getBlockBlobAppendStream(FSDataOutputStream appendStream) {
+    SyncableDataOutputStream dataOutputStream = null;
+
+    if (appendStream.getWrappedStream() instanceof NativeAzureFileSystem.NativeAzureFsOutputStream) {
+      NativeAzureFileSystem.NativeAzureFsOutputStream fsOutputStream =
+              (NativeAzureFileSystem.NativeAzureFsOutputStream) appendStream.getWrappedStream();
+
+      dataOutputStream = (SyncableDataOutputStream) fsOutputStream.getOutStream();
+    }
+
+    if (appendStream.getWrappedStream() instanceof SyncableDataOutputStream) {
+      dataOutputStream = (SyncableDataOutputStream) appendStream.getWrappedStream();
+    }
+
+    Assert.assertNotNull("Did not recognize " + dataOutputStream,
+        dataOutputStream);
+
+    return (BlockBlobAppendStream) dataOutputStream.getOutStream();
+  }
+
+  private void verifyBlockList(BlockBlobAppendStream blockBlobStream,
+                               int[] testData) throws Throwable {
+    List<BlockEntry> blockList = blockBlobStream.getBlockList();
+    Assert.assertEquals("Block list length", testData.length, blockList.size());
+
+    int i = 0;
+    for (BlockEntry block: blockList) {
+      Assert.assertTrue(block.getSize() == testData[i++]);
+    }
+  }
+
+  private void appendBlockList(FSDataOutputStream fsStream,
+                              ByteArrayOutputStream memStream,
+                              int[] testData) throws Throwable {
+
+    for (int d: testData) {
+      byte[] data = getTestData(d);
+      memStream.write(data);
+      fsStream.write(data);
+    }
+    fsStream.hflush();
+  }
+
+  @Test
+  public void testCompactionDisabled() throws Throwable {
+
+    try (FSDataOutputStream appendStream = fs.create(TEST_PATH_NORMAL)) {
+
+      // testing new file
+
+      SyncableDataOutputStream dataOutputStream = null;
+
+      OutputStream wrappedStream = appendStream.getWrappedStream();
+      if (wrappedStream instanceof NativeAzureFileSystem.NativeAzureFsOutputStream) {
+        NativeAzureFileSystem.NativeAzureFsOutputStream fsOutputStream =
+                (NativeAzureFileSystem.NativeAzureFsOutputStream) wrappedStream;
+
+        dataOutputStream = (SyncableDataOutputStream) fsOutputStream.getOutStream();
+      } else if (wrappedStream instanceof SyncableDataOutputStream) {
+        dataOutputStream = (SyncableDataOutputStream) wrappedStream;
+      } else {
+        Assert.fail("Unable to determine type of " + wrappedStream
+            + " class of " + wrappedStream.getClass());
+      }
+
+      Assert.assertFalse("Data output stream is a BlockBlobAppendStream: "
+          + dataOutputStream,
+          dataOutputStream.getOutStream() instanceof BlockBlobAppendStream);
+
+    }
+  }
+
+  @Test
+  public void testCompaction() throws Throwable {
+
+    final int n2 = 2;
+    final int n4 = 4;
+    final int n10 = 10;
+    final int n12 = 12;
+    final int n14 = 14;
+    final int n16 = 16;
+
+    final int maxBlockSize = 16;
+    final int compactionBlockCount = 4;
+
+    ByteArrayOutputStream memStream = new ByteArrayOutputStream();
+
+    try (FSDataOutputStream appendStream = fs.create(TEST_PATH)) {
+
+      // test new file
+
+      BlockBlobAppendStream blockBlobStream = getBlockBlobAppendStream(appendStream);
+      blockBlobStream.setMaxBlockSize(maxBlockSize);
+      blockBlobStream.setCompactionBlockCount(compactionBlockCount);
+
+      appendBlockList(appendStream, memStream, new int[]{n2});
+      verifyBlockList(blockBlobStream, new int[]{n2});
+
+      appendStream.hflush();
+      verifyBlockList(blockBlobStream, new int[]{n2});
+
+      appendBlockList(appendStream, memStream, new int[]{n4});
+      verifyBlockList(blockBlobStream, new int[]{n2, n4});
+
+      appendStream.hsync();
+      verifyBlockList(blockBlobStream, new int[]{n2, n4});
+
+      appendBlockList(appendStream, memStream, new int[]{n4});
+      verifyBlockList(blockBlobStream, new int[]{n2, n4, n4});
+
+      appendBlockList(appendStream, memStream, new int[]{n4});
+      verifyBlockList(blockBlobStream, new int[]{n2, n4, n4, n4});
+
+      appendBlockList(appendStream, memStream, new int[]{n4});
+      verifyBlockList(blockBlobStream, new int[]{n14, n4});
+
+      appendBlockList(appendStream, memStream, new int[]{n4});
+      verifyBlockList(blockBlobStream, new int[]{n14, n4, n4});
+
+      appendBlockList(appendStream, memStream, new int[]{n4});
+      verifyBlockList(blockBlobStream, new int[]{n14, n4, n4, n4});
+
+      appendBlockList(appendStream, memStream, new int[]{n2, n4, n4});
+      verifyBlockList(blockBlobStream, new int[]{n14, n12, n10});
+
+      appendBlockList(appendStream, memStream, new int[]{n4});
+      verifyBlockList(blockBlobStream, new int[]{n14, n12, n10, n4});
+
+      appendBlockList(appendStream, memStream,
+              new int[]{n4, n4, n4, n4});
+      verifyBlockList(blockBlobStream,
+              new int[]{n14, n12, n14, n16});
+
+      appendBlockList(appendStream, memStream,
+              new int[]{n4, n4, n4, n4, n4});
+      verifyBlockList(blockBlobStream,
+              new int[]{n14, n12, n14, n16, n16, n4});
+
+      appendBlockList(appendStream, memStream,
+              new int[]{n4});
+      verifyBlockList(blockBlobStream,
+              new int[]{n14, n12, n14, n16, n16, n4, n4});
+
+      appendBlockList(appendStream, memStream,
+              new int[]{n4});
+      verifyBlockList(blockBlobStream,
+              new int[]{n14, n12, n14, n16, n16, n4, n4, n4});
+
+      appendBlockList(appendStream, memStream,
+              new int[]{n4});
+      verifyBlockList(blockBlobStream,
+              new int[]{n14, n12, n14, n16, n16, n4, n4, n4, n4});
+
+      appendBlockList(appendStream, memStream, new int[]{n4});
+
+      appendStream.close();
+
+      ContractTestUtils.verifyFileContents(fs, TEST_PATH, memStream.toByteArray());
+    }
+
+    try (FSDataOutputStream appendStream = fs.append(TEST_PATH)) {
+
+      // test existing file
+
+      BlockBlobAppendStream blockBlobStream = getBlockBlobAppendStream(appendStream);
+      blockBlobStream.setMaxBlockSize(maxBlockSize);
+      blockBlobStream.setCompactionBlockCount(compactionBlockCount);
+
+      appendBlockList(appendStream, memStream, new int[]{n4});
+      verifyBlockList(blockBlobStream,
+              new int[]{n14, n12, n14, n16, n16, n16, n4, n4});
+
+      appendBlockList(appendStream, memStream, new int[]{n4});
+      verifyBlockList(blockBlobStream,
+              new int[]{n14, n12, n14, n16, n16, n16, n4, n4, n4});
+
+      appendBlockList(appendStream, memStream, new int[]{n4});
+      verifyBlockList(blockBlobStream,
+              new int[]{n14, n12, n14, n16, n16, n16, n4, n4, n4, n4});
+
+      appendBlockList(appendStream, memStream, new int[]{n4});
+      verifyBlockList(blockBlobStream,
+              new int[]{n14, n12, n14, n16, n16, n16, n16, n4});
+
+      appendStream.close();
+
+      ContractTestUtils.verifyFileContents(fs, TEST_PATH, memStream.toByteArray());
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/13eda500/hadoop-tools/hadoop-azure/src/test/resources/log4j.properties
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/resources/log4j.properties b/hadoop-tools/hadoop-azure/src/test/resources/log4j.properties
index 73ee3f9..a5e0c4f 100644
--- a/hadoop-tools/hadoop-azure/src/test/resources/log4j.properties
+++ b/hadoop-tools/hadoop-azure/src/test/resources/log4j.properties
@@ -23,3 +23,4 @@ log4j.appender.stdout.layout=org.apache.log4j.PatternLayout
 log4j.appender.stdout.layout.ConversionPattern=%d{ISO8601} %-5p [%t]: %c{2} (%F:%M(%L)) - %m%n
 
 log4j.logger.org.apache.hadoop.fs.azure.AzureFileSystemThreadPoolExecutor=DEBUG
+log4j.logger.org.apache.hadoop.fs.azure.BlockBlobAppendStream=DEBUG


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[04/37] hadoop git commit: YARN-7075. Better styling for donut charts in new YARN UI. Contributed by Da Ding.

Posted by ae...@apache.org.
YARN-7075. Better styling for donut charts in new YARN UI. Contributed by Da Ding.


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

Branch: refs/heads/HDFS-7240
Commit: 275980bb1e3ae2dd5c64673e3cb384eb104a947d
Parents: 7996eca
Author: Sunil G <su...@apache.org>
Authored: Sat Sep 2 08:08:59 2017 +0530
Committer: Sunil G <su...@apache.org>
Committed: Sat Sep 2 08:08:59 2017 +0530

----------------------------------------------------------------------
 .../main/webapp/app/components/donut-chart.js   |  5 +++-
 .../src/main/webapp/app/styles/app.css          | 24 ++++++++++++++++++--
 2 files changed, 26 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/275980bb/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/components/donut-chart.js
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/components/donut-chart.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/components/donut-chart.js
index 82d2d46..b1e6ecf 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/components/donut-chart.js
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/components/donut-chart.js
@@ -53,7 +53,10 @@ export default BaseChartComponent.extend({
 
     // 50 is for title
     var outerRadius = (h - 50 - 2 * layout.margin) / 2;
-    var innerRadius = outerRadius * 0.618;
+
+    // Ratio of inner radius to outer radius
+    var radiusRatio = 0.75;
+    var innerRadius = outerRadius * radiusRatio;
 
     var arc = d3.svg.arc()
       .innerRadius(innerRadius)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/275980bb/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/styles/app.css
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/styles/app.css b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/styles/app.css
index 8b8ea56..38e25e4 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/styles/app.css
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/styles/app.css
@@ -20,6 +20,8 @@ body, html, body > .ember-view {
   height: 100%;
   overflow: visible;
   color: @text-color;
+  background: #f6f6f8;
+  font-family: "Open Sans","Helvetica Neue", sans-serif!important;
 }
 body, html {
   min-width: 1024px;
@@ -29,11 +31,11 @@ body, html {
  Over all style
  */
 text {
-  font: 16px sans-serif;
+  font-size: 14px;
 }
 
 text.small {
-  font: 8px sans-serif;
+  font-size: 8px;
 }
 
 html, body
@@ -184,6 +186,7 @@ table.dataTable thead .sorting_desc_disabled {
 
 .breadcrumb {
   padding-bottom: 3px;
+  background-color: #f6f6f8;
 }
 
 .navbar-default .navbar-nav > li > a {
@@ -268,12 +271,29 @@ td {
   margin: 0 auto -40px; // Must be same as footer & footer-frame
 }
 
+.panel {
+  background-color: #fff;
+  border: 1px solid #ddd;
+  border-radius: 15px;
+  -webkit-box-shadow: 0 1px 1px rgba(0, 0, 0, 0.05);
+  box-shadow: 0 1px 1px rgba(0, 0, 0, 0.05);
+  overflow: hidden;
+}
+
 .panel-default .container-fluid {
   margin-top: -45px !important;
   margin-bottom: -10px !important;
 }
 
+.panel-default > .panel-heading {
+  background-image: none;
+  background-color: #f5f5f5 !important;
+  border-radius: 15px 15px 0px 0px;
+}
+
+
 .panel-heading {
+  background-color: rgba(0, 0, 0, 0) !important;
   font-weight: bold;
 }
 


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[13/37] hadoop git commit: HDFS-12377. Refactor TestReadStripedFileWithDecoding to avoid test timeouts.

Posted by ae...@apache.org.
HDFS-12377. Refactor TestReadStripedFileWithDecoding to avoid test timeouts.


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

Branch: refs/heads/HDFS-7240
Commit: d4035d42f02507bc89adce3f0450c36b58b201c1
Parents: ad32759
Author: Andrew Wang <wa...@apache.org>
Authored: Tue Sep 5 16:33:29 2017 -0700
Committer: Andrew Wang <wa...@apache.org>
Committed: Tue Sep 5 16:33:29 2017 -0700

----------------------------------------------------------------------
 .../hdfs/ReadStripedFileWithDecodingHelper.java | 273 ++++++++++++++++
 .../apache/hadoop/hdfs/StripedFileTestUtil.java |   9 +-
 .../hdfs/TestReadStripedFileWithDNFailure.java  | 107 +++++++
 .../hdfs/TestReadStripedFileWithDecoding.java   | 309 +++----------------
 ...tReadStripedFileWithDecodingCorruptData.java |  87 ++++++
 ...tReadStripedFileWithDecodingDeletedData.java |  88 ++++++
 6 files changed, 596 insertions(+), 277 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/d4035d42/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/ReadStripedFileWithDecodingHelper.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/ReadStripedFileWithDecodingHelper.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/ReadStripedFileWithDecodingHelper.java
new file mode 100644
index 0000000..4202969
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/ReadStripedFileWithDecodingHelper.java
@@ -0,0 +1,273 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs;
+
+import org.apache.commons.logging.LogFactory;
+import org.apache.commons.logging.impl.Log4JLogger;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.BlockLocation;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
+import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
+import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
+import org.apache.hadoop.hdfs.protocol.LocatedStripedBlock;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockPlacementPolicy;
+import org.apache.hadoop.hdfs.server.datanode.DataNode;
+import org.apache.hadoop.hdfs.server.namenode.NameNode;
+import org.apache.hadoop.hdfs.util.StripedBlockUtil;
+import org.apache.hadoop.test.GenericTestUtils;
+import org.junit.Assert;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.slf4j.event.Level;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Collection;
+
+/**
+ * Utility class for testing online recovery of striped files.
+ */
+abstract public class ReadStripedFileWithDecodingHelper {
+  static final Logger LOG =
+      LoggerFactory.getLogger(ReadStripedFileWithDecodingHelper.class);
+
+  static {
+    ((Log4JLogger)LogFactory.getLog(BlockPlacementPolicy.class))
+        .getLogger().setLevel(org.apache.log4j.Level.ALL);
+    GenericTestUtils.setLogLevel(BlockManager.LOG, Level.DEBUG);
+    GenericTestUtils.setLogLevel(BlockManager.blockLog, Level.DEBUG);
+    GenericTestUtils.setLogLevel(NameNode.stateChangeLog, Level.DEBUG);
+  }
+
+  protected static final ErasureCodingPolicy EC_POLICY =
+      StripedFileTestUtil.getDefaultECPolicy();
+  protected static final short NUM_DATA_UNITS =
+      (short) EC_POLICY.getNumDataUnits();
+  protected static final short NUM_PARITY_UNITS =
+      (short) EC_POLICY.getNumParityUnits();
+  protected static final int CELL_SIZE = EC_POLICY.getCellSize();
+  private static final int STRIPES_PER_BLOCK = 4;
+  protected static final int BLOCK_SIZE = CELL_SIZE * STRIPES_PER_BLOCK;
+  private static final int BLOCK_GROUP_SIZE = BLOCK_SIZE * NUM_DATA_UNITS;
+
+  private static final int NUM_DATANODES = NUM_DATA_UNITS + NUM_PARITY_UNITS;
+
+  protected static final int[] FILE_LENGTHS =
+      {BLOCK_GROUP_SIZE - 123, BLOCK_GROUP_SIZE + 123};
+
+  public static MiniDFSCluster initializeCluster() throws IOException {
+    Configuration conf = new HdfsConfiguration();
+    conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, BLOCK_SIZE);
+    conf.setInt(DFSConfigKeys.DFS_NAMENODE_REPLICATION_MAX_STREAMS_KEY, 0);
+    conf.setInt(DFSConfigKeys.DFS_NAMENODE_REPLICATION_STREAMS_HARD_LIMIT_KEY,
+        0);
+    conf.setBoolean(DFSConfigKeys.DFS_NAMENODE_REDUNDANCY_CONSIDERLOAD_KEY,
+        false);
+    conf.set(DFSConfigKeys.DFS_NAMENODE_EC_POLICIES_ENABLED_KEY,
+        StripedFileTestUtil.getDefaultECPolicy().getName());
+    MiniDFSCluster myCluster = new MiniDFSCluster.Builder(conf)
+        .numDataNodes(NUM_DATANODES)
+        .build();
+    myCluster.getFileSystem().getClient().setErasureCodingPolicy("/",
+        StripedFileTestUtil.getDefaultECPolicy().getName());
+    return myCluster;
+  }
+
+  public static void tearDownCluster(MiniDFSCluster cluster)
+      throws IOException {
+    if (cluster != null) {
+      cluster.shutdown();
+    }
+  }
+
+  public static int findFirstDataNode(MiniDFSCluster cluster,
+      DistributedFileSystem dfs, Path file, long length) throws IOException {
+    BlockLocation[] locs = dfs.getFileBlockLocations(file, 0, length);
+    String name = (locs[0].getNames())[0];
+    int dnIndex = 0;
+    for (DataNode dn : cluster.getDataNodes()) {
+      int port = dn.getXferPort();
+      if (name.contains(Integer.toString(port))) {
+        return dnIndex;
+      }
+      dnIndex++;
+    }
+    return -1;
+  }
+
+  /**
+   * Cross product of FILE_LENGTHS, NUM_PARITY_UNITS+1, NUM_PARITY_UNITS.
+   * Input for parameterized tests classes.
+   *
+   * @return Test parameters.
+   */
+  public static Collection<Object[]> getParameters() {
+    ArrayList<Object[]> params = new ArrayList<>();
+    for (int fileLength : FILE_LENGTHS) {
+      for (int dataDelNum = 1; dataDelNum <= NUM_PARITY_UNITS; dataDelNum++) {
+        for (int parityDelNum = 0;
+             (dataDelNum + parityDelNum) <= NUM_PARITY_UNITS; parityDelNum++) {
+          params.add(new Object[] {fileLength, dataDelNum, parityDelNum});
+        }
+      }
+    }
+    return params;
+  }
+
+  public static void verifyRead(DistributedFileSystem dfs, Path testPath,
+      int length, byte[] expected) throws IOException {
+    LOG.info("verifyRead on path {}", testPath);
+    byte[] buffer = new byte[length + 100];
+    LOG.info("verifyRead verifyLength on path {}", testPath);
+    StripedFileTestUtil.verifyLength(dfs, testPath, length);
+    LOG.info("verifyRead verifyPread on path {}", testPath);
+    StripedFileTestUtil.verifyPread(dfs, testPath, length, expected, buffer);
+    LOG.info("verifyRead verifyStatefulRead on path {}", testPath);
+    StripedFileTestUtil.verifyStatefulRead(dfs, testPath, length, expected,
+        buffer);
+    LOG.info("verifyRead verifyStatefulRead2 on path {}", testPath);
+    StripedFileTestUtil.verifyStatefulRead(dfs, testPath, length, expected,
+        ByteBuffer.allocate(length + 100));
+    LOG.info("verifyRead verifySeek on path {}", testPath);
+    StripedFileTestUtil.verifySeek(dfs, testPath, length, EC_POLICY,
+        BLOCK_GROUP_SIZE);
+  }
+
+  public static void testReadWithDNFailure(MiniDFSCluster cluster,
+      DistributedFileSystem dfs, int fileLength, int dnFailureNum)
+      throws Exception {
+    String fileType = fileLength < (BLOCK_SIZE * NUM_DATA_UNITS) ?
+        "smallFile" : "largeFile";
+    String src = "/dnFailure_" + dnFailureNum + "_" + fileType;
+    LOG.info("testReadWithDNFailure: file = " + src
+        + ", fileSize = " + fileLength
+        + ", dnFailureNum = " + dnFailureNum);
+
+    Path testPath = new Path(src);
+    final byte[] bytes = StripedFileTestUtil.generateBytes(fileLength);
+    DFSTestUtil.writeFile(dfs, testPath, bytes);
+    StripedFileTestUtil.waitBlockGroupsReported(dfs, src);
+
+    // shut down the DN that holds an internal data block
+    BlockLocation[] locs = dfs.getFileBlockLocations(testPath, CELL_SIZE * 5,
+        CELL_SIZE);
+    for (int failedDnIdx = 0; failedDnIdx < dnFailureNum; failedDnIdx++) {
+      String name = (locs[0].getNames())[failedDnIdx];
+      for (DataNode dn : cluster.getDataNodes()) {
+        int port = dn.getXferPort();
+        if (name.contains(Integer.toString(port))) {
+          dn.shutdown();
+        }
+      }
+    }
+
+    // check file length, pread, stateful read and seek
+    verifyRead(dfs, testPath, fileLength, bytes);
+  }
+
+
+  /**
+   * Test reading a file with some blocks(data blocks or parity blocks or both)
+   * deleted or corrupted.
+   * @param src file path
+   * @param fileNumBytes file length
+   * @param dataBlkDelNum the deleted or corrupted number of data blocks.
+   * @param parityBlkDelNum the deleted or corrupted number of parity blocks.
+   * @param deleteBlockFile whether block file is deleted or corrupted.
+   *                        true is to delete the block file.
+   *                        false is to corrupt the content of the block file.
+   * @throws IOException
+   */
+  public static void testReadWithBlockCorrupted(MiniDFSCluster cluster,
+      DistributedFileSystem dfs, String src, int fileNumBytes,
+      int dataBlkDelNum, int parityBlkDelNum,
+      boolean deleteBlockFile) throws IOException {
+    LOG.info("testReadWithBlockCorrupted: file = " + src
+        + ", dataBlkDelNum = " + dataBlkDelNum
+        + ", parityBlkDelNum = " + parityBlkDelNum
+        + ", deleteBlockFile? " + deleteBlockFile);
+    int recoverBlkNum = dataBlkDelNum + parityBlkDelNum;
+    Assert.assertTrue("dataBlkDelNum and parityBlkDelNum should be positive",
+        dataBlkDelNum >= 0 && parityBlkDelNum >= 0);
+    Assert.assertTrue("The sum of dataBlkDelNum and parityBlkDelNum " +
+        "should be between 1 ~ " + NUM_PARITY_UNITS, recoverBlkNum <=
+        NUM_PARITY_UNITS);
+
+    // write a file with the length of writeLen
+    Path srcPath = new Path(src);
+    final byte[] bytes = StripedFileTestUtil.generateBytes(fileNumBytes);
+    DFSTestUtil.writeFile(dfs, srcPath, bytes);
+
+    // delete or corrupt some blocks
+    corruptBlocks(cluster, dfs, srcPath, dataBlkDelNum, parityBlkDelNum,
+        deleteBlockFile);
+
+    // check the file can be read after some blocks were deleted
+    verifyRead(dfs, srcPath, fileNumBytes, bytes);
+  }
+
+  public static void corruptBlocks(MiniDFSCluster cluster,
+      DistributedFileSystem dfs, Path srcPath,
+      int dataBlkDelNum, int parityBlkDelNum, boolean deleteBlockFile)
+      throws IOException {
+    LOG.info("corruptBlocks on path {}", srcPath);
+    int recoverBlkNum = dataBlkDelNum + parityBlkDelNum;
+
+    LocatedBlocks locatedBlocks = getLocatedBlocks(dfs, srcPath);
+    LocatedStripedBlock lastBlock =
+        (LocatedStripedBlock)locatedBlocks.getLastLocatedBlock();
+
+    int[] delDataBlkIndices = StripedFileTestUtil.randomArray(0, NUM_DATA_UNITS,
+        dataBlkDelNum);
+    Assert.assertNotNull(delDataBlkIndices);
+    int[] delParityBlkIndices = StripedFileTestUtil.randomArray(NUM_DATA_UNITS,
+        NUM_DATA_UNITS + NUM_PARITY_UNITS, parityBlkDelNum);
+    Assert.assertNotNull(delParityBlkIndices);
+
+    int[] delBlkIndices = new int[recoverBlkNum];
+    System.arraycopy(delDataBlkIndices, 0,
+        delBlkIndices, 0, delDataBlkIndices.length);
+    System.arraycopy(delParityBlkIndices, 0,
+        delBlkIndices, delDataBlkIndices.length, delParityBlkIndices.length);
+
+    ExtendedBlock[] delBlocks = new ExtendedBlock[recoverBlkNum];
+    for (int i = 0; i < recoverBlkNum; i++) {
+      delBlocks[i] = StripedBlockUtil
+          .constructInternalBlock(lastBlock.getBlock(),
+              CELL_SIZE, NUM_DATA_UNITS, delBlkIndices[i]);
+      if (deleteBlockFile) {
+        // delete the block file
+        LOG.info("Deleting block file {}", delBlocks[i]);
+        cluster.corruptBlockOnDataNodesByDeletingBlockFile(delBlocks[i]);
+      } else {
+        // corrupt the block file
+        LOG.info("Corrupting block file {}", delBlocks[i]);
+        cluster.corruptBlockOnDataNodes(delBlocks[i]);
+      }
+    }
+  }
+
+  public static LocatedBlocks getLocatedBlocks(DistributedFileSystem dfs,
+      Path filePath) throws IOException {
+    return dfs.getClient().getLocatedBlocks(filePath.toString(),
+        0, Long.MAX_VALUE);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d4035d42/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/StripedFileTestUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/StripedFileTestUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/StripedFileTestUtil.java
index 057e94a..1489e48 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/StripedFileTestUtil.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/StripedFileTestUtil.java
@@ -18,8 +18,6 @@
 package org.apache.hadoop.hdfs;
 
 import com.google.common.base.Joiner;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FileStatus;
@@ -27,12 +25,12 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.client.impl.BlockReaderTestUtil;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
-import org.apache.hadoop.hdfs.protocol.SystemErasureCodingPolicies;
 import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.protocol.LocatedStripedBlock;
+import org.apache.hadoop.hdfs.protocol.SystemErasureCodingPolicies;
 import org.apache.hadoop.hdfs.util.StripedBlockUtil;
 import org.apache.hadoop.hdfs.web.WebHdfsFileSystem.WebHdfsInputStream;
 import org.apache.hadoop.io.IOUtils;
@@ -40,6 +38,8 @@ import org.apache.hadoop.io.erasurecode.CodecUtil;
 import org.apache.hadoop.io.erasurecode.ErasureCoderOptions;
 import org.apache.hadoop.io.erasurecode.rawcoder.RawErasureEncoder;
 import org.junit.Assert;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import java.io.EOFException;
 import java.io.IOException;
@@ -57,7 +57,8 @@ import java.util.concurrent.atomic.AtomicInteger;
 import static org.junit.Assert.assertEquals;
 
 public class StripedFileTestUtil {
-  public static final Log LOG = LogFactory.getLog(StripedFileTestUtil.class);
+  public static final Logger LOG =
+      LoggerFactory.getLogger(StripedFileTestUtil.class);
 
   public static byte[] generateBytes(int cnt) {
     byte[] bytes = new byte[cnt];

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d4035d42/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadStripedFileWithDNFailure.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadStripedFileWithDNFailure.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadStripedFileWithDNFailure.java
new file mode 100644
index 0000000..40ac206
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadStripedFileWithDNFailure.java
@@ -0,0 +1,107 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs;
+
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.Timeout;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+
+import static org.apache.hadoop.hdfs.ReadStripedFileWithDecodingHelper.BLOCK_SIZE;
+import static org.apache.hadoop.hdfs.ReadStripedFileWithDecodingHelper.FILE_LENGTHS;
+import static org.apache.hadoop.hdfs.ReadStripedFileWithDecodingHelper.NUM_DATA_UNITS;
+import static org.apache.hadoop.hdfs.ReadStripedFileWithDecodingHelper.NUM_PARITY_UNITS;
+import static org.apache.hadoop.hdfs.ReadStripedFileWithDecodingHelper.initializeCluster;
+import static org.apache.hadoop.hdfs.ReadStripedFileWithDecodingHelper.tearDownCluster;
+
+/**
+ * Test online recovery with failed DNs. This test is parameterized.
+ */
+@RunWith(Parameterized.class)
+public class TestReadStripedFileWithDNFailure {
+  static final Logger LOG =
+      LoggerFactory.getLogger(TestReadStripedFileWithDNFailure.class);
+
+  private static MiniDFSCluster cluster;
+  private static DistributedFileSystem dfs;
+
+  @Rule
+  public Timeout globalTimeout = new Timeout(300000);
+
+  @BeforeClass
+  public static void setup() throws IOException {
+    cluster = initializeCluster();
+    dfs = cluster.getFileSystem();
+  }
+
+  @AfterClass
+  public static void tearDown() throws IOException {
+    tearDownCluster(cluster);
+  }
+
+  @Parameterized.Parameters
+  public static Collection<Object[]> getParameters() {
+    ArrayList<Object[]> params = new ArrayList<>();
+    for (int fileLength : FILE_LENGTHS) {
+      for (int i = 0; i < NUM_PARITY_UNITS; i++) {
+        params.add(new Object[] {fileLength, i+1});
+      }
+    }
+    return params;
+  }
+
+  private int fileLength;
+  private int dnFailureNum;
+
+  public TestReadStripedFileWithDNFailure(int fileLength, int dnFailureNum) {
+    this.fileLength = fileLength;
+    this.dnFailureNum = dnFailureNum;
+  }
+
+  /**
+   * Shutdown tolerable number of Datanode before reading.
+   * Verify the decoding works correctly.
+   */
+  @Test
+  public void testReadWithDNFailure() throws Exception {
+    try {
+      // setup a new cluster with no dead datanode
+      setup();
+      ReadStripedFileWithDecodingHelper.testReadWithDNFailure(cluster,
+          dfs, fileLength, dnFailureNum);
+    } catch (IOException ioe) {
+      String fileType = fileLength < (BLOCK_SIZE * NUM_DATA_UNITS) ?
+          "smallFile" : "largeFile";
+      LOG.error("Failed to read file with DN failure:"
+          + " fileType = " + fileType
+          + ", dnFailureNum = " + dnFailureNum);
+    } finally {
+      // tear down the cluster
+      tearDown();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d4035d42/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadStripedFileWithDecoding.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadStripedFileWithDecoding.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadStripedFileWithDecoding.java
index cb1640c..2fb9212 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadStripedFileWithDecoding.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadStripedFileWithDecoding.java
@@ -17,222 +17,58 @@
  */
 package org.apache.hadoop.hdfs;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.commons.logging.impl.Log4JLogger;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.BlockLocation;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.protocol.Block;
-import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
-import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
-import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.protocol.LocatedStripedBlock;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockPlacementPolicy;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
 import org.apache.hadoop.hdfs.server.datanode.DataNodeTestUtils;
 import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
-import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.hdfs.server.namenode.NameNodeAdapter;
 import org.apache.hadoop.hdfs.util.StripedBlockUtil;
-import org.apache.hadoop.test.GenericTestUtils;
-import org.apache.log4j.Level;
 import org.junit.After;
 import org.junit.Assert;
 import org.junit.Before;
-import org.junit.Test;
 import org.junit.Rule;
+import org.junit.Test;
 import org.junit.rules.Timeout;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import java.io.File;
 import java.io.FileOutputStream;
 import java.io.IOException;
 import java.nio.ByteBuffer;
 
-public class TestReadStripedFileWithDecoding {
-  static final Log LOG = LogFactory.getLog(TestReadStripedFileWithDecoding.class);
+import static org.apache.hadoop.hdfs.ReadStripedFileWithDecodingHelper.CELL_SIZE;
+import static org.apache.hadoop.hdfs.ReadStripedFileWithDecodingHelper.NUM_DATA_UNITS;
+import static org.apache.hadoop.hdfs.ReadStripedFileWithDecodingHelper.NUM_PARITY_UNITS;
+import static org.apache.hadoop.hdfs.ReadStripedFileWithDecodingHelper.findFirstDataNode;
+import static org.apache.hadoop.hdfs.ReadStripedFileWithDecodingHelper.initializeCluster;
+import static org.apache.hadoop.hdfs.ReadStripedFileWithDecodingHelper.tearDownCluster;
 
-  static {
-    ((Log4JLogger)LogFactory.getLog(BlockPlacementPolicy.class))
-        .getLogger().setLevel(Level.ALL);
-    GenericTestUtils.setLogLevel(BlockManager.LOG, Level.ALL);
-    GenericTestUtils.setLogLevel(BlockManager.blockLog, Level.ALL);
-    GenericTestUtils.setLogLevel(NameNode.stateChangeLog, Level.ALL);
-  }
+public class TestReadStripedFileWithDecoding {
+  private static final Logger LOG =
+      LoggerFactory.getLogger(TestReadStripedFileWithDecoding.class);
 
   private MiniDFSCluster cluster;
-  private DistributedFileSystem fs;
-  private final ErasureCodingPolicy ecPolicy =
-      StripedFileTestUtil.getDefaultECPolicy();
-  private final short dataBlocks = (short) ecPolicy.getNumDataUnits();
-  private final short parityBlocks =
-      (short) ecPolicy.getNumParityUnits();
-  private final int numDNs = dataBlocks + parityBlocks;
-  private final int cellSize = ecPolicy.getCellSize();
-  private final int stripPerBlock = 4;
-  private final int blockSize = cellSize * stripPerBlock;
-  private final int blockGroupSize = blockSize * dataBlocks;
-  private final int smallFileLength = blockGroupSize - 123;
-  private final int largeFileLength = blockGroupSize + 123;
-  private final int[] fileLengths = {smallFileLength, largeFileLength};
-  private final int[] dnFailureNums = getDnFailureNums();
-
-  private int[] getDnFailureNums() {
-    int[] dnFailureNums = new int[parityBlocks];
-    for (int i = 0; i < dnFailureNums.length; i++) {
-      dnFailureNums[i] = i + 1;
-    }
-    return dnFailureNums;
-  }
+  private DistributedFileSystem dfs;
 
   @Rule
   public Timeout globalTimeout = new Timeout(300000);
 
   @Before
   public void setup() throws IOException {
-    Configuration conf = new HdfsConfiguration();
-    conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, blockSize);
-    conf.setInt(DFSConfigKeys.DFS_NAMENODE_REPLICATION_MAX_STREAMS_KEY, 0);
-    conf.setInt(DFSConfigKeys.DFS_NAMENODE_REPLICATION_STREAMS_HARD_LIMIT_KEY, 0);
-    conf.setBoolean(DFSConfigKeys.DFS_NAMENODE_REDUNDANCY_CONSIDERLOAD_KEY,
-        false);
-    conf.set(DFSConfigKeys.DFS_NAMENODE_EC_POLICIES_ENABLED_KEY,
-        StripedFileTestUtil.getDefaultECPolicy().getName());
-    cluster = new MiniDFSCluster.Builder(conf).numDataNodes(numDNs).build();
-    cluster.getFileSystem().getClient().setErasureCodingPolicy("/",
-        StripedFileTestUtil.getDefaultECPolicy().getName());
-    fs = cluster.getFileSystem();
+    cluster = initializeCluster();
+    dfs = cluster.getFileSystem();
   }
 
   @After
   public void tearDown() throws IOException {
-    if (cluster != null) {
-      cluster.shutdown();
-      cluster = null;
-    }
-  }
-
-  /**
-   * Shutdown tolerable number of Datanode before reading.
-   * Verify the decoding works correctly.
-   */
-  @Test(timeout=300000)
-  public void testReadWithDNFailure() throws Exception {
-    for (int fileLength : fileLengths) {
-      for (int dnFailureNum : dnFailureNums) {
-        try {
-          // setup a new cluster with no dead datanode
-          setup();
-          testReadWithDNFailure(fileLength, dnFailureNum);
-        } catch (IOException ioe) {
-          String fileType = fileLength < (blockSize * dataBlocks) ?
-              "smallFile" : "largeFile";
-          LOG.error("Failed to read file with DN failure:"
-              + " fileType = "+ fileType
-              + ", dnFailureNum = " + dnFailureNum);
-        } finally {
-          // tear down the cluster
-          tearDown();
-        }
-      }
-    }
-  }
-
-  /**
-   * Corrupt tolerable number of block before reading.
-   * Verify the decoding works correctly.
-   */
-  @Test(timeout=300000)
-  public void testReadCorruptedData() throws IOException {
-    for (int fileLength : fileLengths) {
-      for (int dataDelNum = 1; dataDelNum <= parityBlocks; dataDelNum++) {
-        for (int parityDelNum = 0; (dataDelNum + parityDelNum) <= parityBlocks;
-             parityDelNum++) {
-          String src = "/corrupted_" + dataDelNum + "_" + parityDelNum;
-          testReadWithBlockCorrupted(src, fileLength,
-              dataDelNum, parityDelNum, false);
-        }
-      }
-    }
-  }
-
-  /**
-   * Delete tolerable number of block before reading.
-   * Verify the decoding works correctly.
-   */
-  @Test(timeout=300000)
-  public void testReadCorruptedDataByDeleting() throws IOException {
-    for (int fileLength : fileLengths) {
-      for (int dataDelNum = 1; dataDelNum <= parityBlocks; dataDelNum++) {
-        for (int parityDelNum = 0; (dataDelNum + parityDelNum) <= parityBlocks;
-             parityDelNum++) {
-          String src = "/deleted_" + dataDelNum + "_" + parityDelNum;
-          testReadWithBlockCorrupted(src, fileLength,
-              dataDelNum, parityDelNum, true);
-        }
-      }
-    }
-  }
-
-  private int findFirstDataNode(Path file, long length) throws IOException {
-    BlockLocation[] locs = fs.getFileBlockLocations(file, 0, length);
-    String name = (locs[0].getNames())[0];
-    int dnIndex = 0;
-    for (DataNode dn : cluster.getDataNodes()) {
-      int port = dn.getXferPort();
-      if (name.contains(Integer.toString(port))) {
-        return dnIndex;
-      }
-      dnIndex++;
-    }
-    return -1;
-  }
-
-  private void verifyRead(Path testPath, int length, byte[] expected)
-      throws IOException {
-    byte[] buffer = new byte[length + 100];
-    StripedFileTestUtil.verifyLength(fs, testPath, length);
-    StripedFileTestUtil.verifyPread(fs, testPath, length, expected, buffer);
-    StripedFileTestUtil.verifyStatefulRead(fs, testPath, length, expected, buffer);
-    StripedFileTestUtil.verifyStatefulRead(fs, testPath, length, expected,
-        ByteBuffer.allocate(length + 100));
-    StripedFileTestUtil.verifySeek(fs, testPath, length, ecPolicy,
-        blockGroupSize);
-  }
-
-  private void testReadWithDNFailure(int fileLength, int dnFailureNum)
-      throws Exception {
-    String fileType = fileLength < (blockSize * dataBlocks) ?
-        "smallFile" : "largeFile";
-    String src = "/dnFailure_" + dnFailureNum + "_" + fileType;
-    LOG.info("testReadWithDNFailure: file = " + src
-        + ", fileSize = " + fileLength
-        + ", dnFailureNum = " + dnFailureNum);
-
-    Path testPath = new Path(src);
-    final byte[] bytes = StripedFileTestUtil.generateBytes(fileLength);
-    DFSTestUtil.writeFile(fs, testPath, bytes);
-    StripedFileTestUtil.waitBlockGroupsReported(fs, src);
-
-    // shut down the DN that holds an internal data block
-    BlockLocation[] locs = fs.getFileBlockLocations(testPath, cellSize * 5,
-        cellSize);
-    for (int failedDnIdx = 0; failedDnIdx < dnFailureNum; failedDnIdx++) {
-      String name = (locs[0].getNames())[failedDnIdx];
-      for (DataNode dn : cluster.getDataNodes()) {
-        int port = dn.getXferPort();
-        if (name.contains(Integer.toString(port))) {
-          dn.shutdown();
-        }
-      }
-    }
-
-    // check file length, pread, stateful read and seek
-    verifyRead(testPath, fileLength, bytes);
+    tearDownCluster(cluster);
   }
 
   /**
@@ -245,15 +81,17 @@ public class TestReadStripedFileWithDecoding {
     final Path file = new Path("/corrupted");
     final int length = 10; // length of "corruption"
     final byte[] bytes = StripedFileTestUtil.generateBytes(length);
-    DFSTestUtil.writeFile(fs, file, bytes);
+    DFSTestUtil.writeFile(dfs, file, bytes);
 
     // corrupt the first data block
-    int dnIndex = findFirstDataNode(file, cellSize * dataBlocks);
+    int dnIndex = ReadStripedFileWithDecodingHelper.findFirstDataNode(
+        cluster, dfs, file, CELL_SIZE * NUM_DATA_UNITS);
     Assert.assertNotEquals(-1, dnIndex);
-    LocatedStripedBlock slb = (LocatedStripedBlock)fs.getClient()
-        .getLocatedBlocks(file.toString(), 0, cellSize * dataBlocks).get(0);
+    LocatedStripedBlock slb = (LocatedStripedBlock) dfs.getClient()
+        .getLocatedBlocks(file.toString(), 0, CELL_SIZE * NUM_DATA_UNITS)
+        .get(0);
     final LocatedBlock[] blks = StripedBlockUtil.parseStripedBlockGroup(slb,
-        cellSize, dataBlocks, parityBlocks);
+        CELL_SIZE, NUM_DATA_UNITS, NUM_PARITY_UNITS);
     // find the first block file
     File storageDir = cluster.getInstanceStorageDir(dnIndex, 0);
     File blkFile = MiniDFSCluster.getBlockFile(storageDir, blks[0].getBlock());
@@ -272,7 +110,7 @@ public class TestReadStripedFileWithDecoding {
 
     try {
       // do stateful read
-      StripedFileTestUtil.verifyStatefulRead(fs, file, length, bytes,
+      StripedFileTestUtil.verifyStatefulRead(dfs, file, length, bytes,
           ByteBuffer.allocate(1024));
 
       // check whether the corruption has been reported to the NameNode
@@ -293,110 +131,35 @@ public class TestReadStripedFileWithDecoding {
     final Path file = new Path("/invalidate");
     final int length = 10;
     final byte[] bytes = StripedFileTestUtil.generateBytes(length);
-    DFSTestUtil.writeFile(fs, file, bytes);
+    DFSTestUtil.writeFile(dfs, file, bytes);
 
-    int dnIndex = findFirstDataNode(file, cellSize * dataBlocks);
+    int dnIndex = findFirstDataNode(cluster, dfs, file,
+        CELL_SIZE * NUM_DATA_UNITS);
     Assert.assertNotEquals(-1, dnIndex);
-    LocatedStripedBlock slb = (LocatedStripedBlock)fs.getClient()
-        .getLocatedBlocks(file.toString(), 0, cellSize * dataBlocks).get(0);
+    LocatedStripedBlock slb = (LocatedStripedBlock) dfs.getClient()
+        .getLocatedBlocks(file.toString(), 0, CELL_SIZE * NUM_DATA_UNITS)
+        .get(0);
     final LocatedBlock[] blks = StripedBlockUtil.parseStripedBlockGroup(slb,
-        cellSize, dataBlocks, parityBlocks);
+        CELL_SIZE, NUM_DATA_UNITS, NUM_PARITY_UNITS);
     final Block b = blks[0].getBlock().getLocalBlock();
 
     DataNode dn = cluster.getDataNodes().get(dnIndex);
-    // disable the heartbeat from DN so that the invalidated block record is kept
-    // in NameNode until heartbeat expires and NN mark the dn as dead
+    // disable the heartbeat from DN so that the invalidated block record is
+    // kept in NameNode until heartbeat expires and NN mark the dn as dead
     DataNodeTestUtils.setHeartbeatsDisabledForTests(dn, true);
 
     try {
       // delete the file
-      fs.delete(file, true);
+      dfs.delete(file, true);
       // check the block is added to invalidateBlocks
       final FSNamesystem fsn = cluster.getNamesystem();
       final BlockManager bm = fsn.getBlockManager();
-      DatanodeDescriptor dnd = NameNodeAdapter.getDatanode(fsn, dn.getDatanodeId());
+      DatanodeDescriptor dnd =
+          NameNodeAdapter.getDatanode(fsn, dn.getDatanodeId());
       Assert.assertTrue(bm.containsInvalidateBlock(
           blks[0].getLocations()[0], b) || dnd.containsInvalidateBlock(b));
     } finally {
       DataNodeTestUtils.setHeartbeatsDisabledForTests(dn, false);
     }
   }
-
-  /**
-   * Test reading a file with some blocks(data blocks or parity blocks or both)
-   * deleted or corrupted.
-   * @param src file path
-   * @param fileLength file length
-   * @param dataBlkDelNum the deleted or corrupted number of data blocks.
-   * @param parityBlkDelNum the deleted or corrupted number of parity blocks.
-   * @param deleteBlockFile whether block file is deleted or corrupted.
-   *                        true is to delete the block file.
-   *                        false is to corrupt the content of the block file.
-   * @throws IOException
-   */
-  private void testReadWithBlockCorrupted(String src, int fileLength,
-      int dataBlkDelNum, int parityBlkDelNum, boolean deleteBlockFile)
-      throws IOException {
-    LOG.info("testReadWithBlockCorrupted: file = " + src
-        + ", dataBlkDelNum = " + dataBlkDelNum
-        + ", parityBlkDelNum = " + parityBlkDelNum
-        + ", deleteBlockFile? " + deleteBlockFile);
-    int recoverBlkNum = dataBlkDelNum + parityBlkDelNum;
-    Assert.assertTrue("dataBlkDelNum and parityBlkDelNum should be positive",
-        dataBlkDelNum >= 0 && parityBlkDelNum >= 0);
-    Assert.assertTrue("The sum of dataBlkDelNum and parityBlkDelNum " +
-        "should be between 1 ~ " + parityBlocks, recoverBlkNum <= parityBlocks);
-
-    // write a file with the length of writeLen
-    Path srcPath = new Path(src);
-    final byte[] bytes = StripedFileTestUtil.generateBytes(fileLength);
-    DFSTestUtil.writeFile(fs, srcPath, bytes);
-
-    // delete or corrupt some blocks
-    corruptBlocks(srcPath, dataBlkDelNum, parityBlkDelNum, deleteBlockFile);
-
-    // check the file can be read after some blocks were deleted
-    verifyRead(srcPath, fileLength, bytes);
-  }
-
-  private void corruptBlocks(Path srcPath, int dataBlkDelNum,
-      int parityBlkDelNum, boolean deleteBlockFile) throws IOException {
-    int recoverBlkNum = dataBlkDelNum + parityBlkDelNum;
-
-    LocatedBlocks locatedBlocks = getLocatedBlocks(srcPath);
-    LocatedStripedBlock lastBlock =
-        (LocatedStripedBlock)locatedBlocks.getLastLocatedBlock();
-
-    int[] delDataBlkIndices = StripedFileTestUtil.randomArray(0, dataBlocks,
-        dataBlkDelNum);
-    Assert.assertNotNull(delDataBlkIndices);
-    int[] delParityBlkIndices = StripedFileTestUtil.randomArray(dataBlocks,
-        dataBlocks + parityBlocks, parityBlkDelNum);
-    Assert.assertNotNull(delParityBlkIndices);
-
-    int[] delBlkIndices = new int[recoverBlkNum];
-    System.arraycopy(delDataBlkIndices, 0,
-        delBlkIndices, 0, delDataBlkIndices.length);
-    System.arraycopy(delParityBlkIndices, 0,
-        delBlkIndices, delDataBlkIndices.length, delParityBlkIndices.length);
-
-    ExtendedBlock[] delBlocks = new ExtendedBlock[recoverBlkNum];
-    for (int i = 0; i < recoverBlkNum; i++) {
-      delBlocks[i] = StripedBlockUtil
-          .constructInternalBlock(lastBlock.getBlock(),
-              cellSize, dataBlocks, delBlkIndices[i]);
-      if (deleteBlockFile) {
-        // delete the block file
-        cluster.corruptBlockOnDataNodesByDeletingBlockFile(delBlocks[i]);
-      } else {
-        // corrupt the block file
-        cluster.corruptBlockOnDataNodes(delBlocks[i]);
-      }
-    }
-  }
-
-  private LocatedBlocks getLocatedBlocks(Path filePath) throws IOException {
-    return fs.getClient().getLocatedBlocks(filePath.toString(),
-        0, Long.MAX_VALUE);
-  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d4035d42/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadStripedFileWithDecodingCorruptData.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadStripedFileWithDecodingCorruptData.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadStripedFileWithDecodingCorruptData.java
new file mode 100644
index 0000000..5a8fb4f
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadStripedFileWithDecodingCorruptData.java
@@ -0,0 +1,87 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs;
+
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.Timeout;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.Collection;
+
+import static org.apache.hadoop.hdfs.ReadStripedFileWithDecodingHelper.initializeCluster;
+import static org.apache.hadoop.hdfs.ReadStripedFileWithDecodingHelper.tearDownCluster;
+
+/**
+ * Test online recovery with corrupt files. This test is parameterized.
+ */
+@RunWith(Parameterized.class)
+public class TestReadStripedFileWithDecodingCorruptData {
+  static final Logger LOG =
+      LoggerFactory.getLogger(TestReadStripedFileWithDecodingCorruptData.class);
+
+  private static MiniDFSCluster cluster;
+  private static DistributedFileSystem dfs;
+
+  @Rule
+  public Timeout globalTimeout = new Timeout(300000);
+
+  @BeforeClass
+  public static void setup() throws IOException {
+    cluster = initializeCluster();
+    dfs = cluster.getFileSystem();
+  }
+
+  @AfterClass
+  public static void tearDown() throws IOException {
+    tearDownCluster(cluster);
+  }
+
+  @Parameterized.Parameters
+  public static Collection<Object[]> getParameters() {
+    return ReadStripedFileWithDecodingHelper.getParameters();
+  }
+
+  private int fileLength;
+  private int dataDelNum;
+  private int parityDelNum;
+
+  public TestReadStripedFileWithDecodingCorruptData(int fileLength, int
+      dataDelNum, int parityDelNum) {
+    this.fileLength = fileLength;
+    this.dataDelNum = dataDelNum;
+    this.parityDelNum = parityDelNum;
+  }
+
+  /**
+   * Corrupt tolerable number of block before reading.
+   * Verify the decoding works correctly.
+   */
+  @Test
+  public void testReadCorruptedData() throws IOException {
+    String src = "/corrupted_" + dataDelNum + "_" + parityDelNum;
+    ReadStripedFileWithDecodingHelper.testReadWithBlockCorrupted(cluster,
+        dfs, src, fileLength, dataDelNum, parityDelNum, false);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d4035d42/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadStripedFileWithDecodingDeletedData.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadStripedFileWithDecodingDeletedData.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadStripedFileWithDecodingDeletedData.java
new file mode 100644
index 0000000..c267e84
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadStripedFileWithDecodingDeletedData.java
@@ -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.
+ */
+package org.apache.hadoop.hdfs;
+
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.Timeout;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.Collection;
+
+import static org.apache.hadoop.hdfs.ReadStripedFileWithDecodingHelper.initializeCluster;
+import static org.apache.hadoop.hdfs.ReadStripedFileWithDecodingHelper.tearDownCluster;
+
+/**
+ * Test online recovery with files with deleted blocks. This test is
+ * parameterized.
+ */
+@RunWith(Parameterized.class)
+public class TestReadStripedFileWithDecodingDeletedData {
+  static final Logger LOG =
+      LoggerFactory.getLogger(TestReadStripedFileWithDecodingDeletedData.class);
+
+  private static MiniDFSCluster cluster;
+  private static DistributedFileSystem dfs;
+
+  @Rule
+  public Timeout globalTimeout = new Timeout(300000);
+
+  @BeforeClass
+  public static void setup() throws IOException {
+    cluster = initializeCluster();
+    dfs = cluster.getFileSystem();
+  }
+
+  @AfterClass
+  public static void tearDown() throws IOException {
+    tearDownCluster(cluster);
+  }
+
+  @Parameterized.Parameters
+  public static Collection<Object[]> getParameters() {
+    return ReadStripedFileWithDecodingHelper.getParameters();
+  }
+
+  private int fileLength;
+  private int dataDelNum;
+  private int parityDelNum;
+
+  public TestReadStripedFileWithDecodingDeletedData(int fileLength, int
+      dataDelNum, int parityDelNum) {
+    this.fileLength = fileLength;
+    this.dataDelNum = dataDelNum;
+    this.parityDelNum = parityDelNum;
+  }
+
+  /**
+   * Delete tolerable number of block before reading.
+   * Verify the decoding works correctly.
+   */
+  @Test
+  public void testReadCorruptedDataByDeleting() throws IOException {
+    String src = "/deleted_" + dataDelNum + "_" + parityDelNum;
+    ReadStripedFileWithDecodingHelper.testReadWithBlockCorrupted(cluster,
+        dfs, src, fileLength, dataDelNum, parityDelNum, true);
+  }
+}


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[20/37] hadoop git commit: HADOOP-14827. Allow StopWatch to accept a Timer parameter for tests. Contributed by Erik Krogen

Posted by ae...@apache.org.
HADOOP-14827. Allow StopWatch to accept a Timer parameter for tests. Contributed by Erik Krogen


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

Branch: refs/heads/HDFS-7240
Commit: dd814946f68d52a9b1627ac4dd61f9ab093423ae
Parents: 704267c
Author: Jason Lowe <jl...@apache.org>
Authored: Wed Sep 6 16:04:30 2017 -0500
Committer: Jason Lowe <jl...@apache.org>
Committed: Wed Sep 6 16:04:30 2017 -0500

----------------------------------------------------------------------
 .../java/org/apache/hadoop/util/StopWatch.java     | 17 ++++++++++++++---
 1 file changed, 14 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/dd814946/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/StopWatch.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/StopWatch.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/StopWatch.java
index b9d0d0b..c0eedf6 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/StopWatch.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/StopWatch.java
@@ -25,11 +25,22 @@ import java.util.concurrent.TimeUnit;
  * A simplified StopWatch implementation which can measure times in nanoseconds.
  */
 public class StopWatch implements Closeable {
+  private final Timer timer;
   private boolean isStarted;
   private long startNanos;
   private long currentElapsedNanos;
 
   public StopWatch() {
+    this(new Timer());
+  }
+
+  /**
+   * Used for tests to be able to create a StopWatch which does not follow real
+   * time.
+   * @param timer The timer to base this StopWatch's timekeeping off of.
+   */
+  public StopWatch(Timer timer) {
+    this.timer = timer;
   }
 
   /**
@@ -49,7 +60,7 @@ public class StopWatch implements Closeable {
       throw new IllegalStateException("StopWatch is already running");
     }
     isStarted = true;
-    startNanos = System.nanoTime();
+    startNanos = timer.monotonicNowNanos();
     return this;
   }
 
@@ -61,7 +72,7 @@ public class StopWatch implements Closeable {
     if (!isStarted) {
       throw new IllegalStateException("StopWatch is already stopped");
     }
-    long now = System.nanoTime();
+    long now = timer.monotonicNowNanos();
     isStarted = false;
     currentElapsedNanos += now - startNanos;
     return this;
@@ -90,7 +101,7 @@ public class StopWatch implements Closeable {
    */
   public long now() {
     return isStarted ?
-        System.nanoTime() - startNanos + currentElapsedNanos :
+        timer.monotonicNowNanos() - startNanos + currentElapsedNanos :
         currentElapsedNanos;
   }
 


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[34/37] hadoop git commit: HADOOP-14520. WASB: Block compaction for Azure Block Blobs. Contributed by Georgi Chalakov

Posted by ae...@apache.org.
HADOOP-14520. WASB: Block compaction for Azure Block Blobs.
Contributed by Georgi Chalakov


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

Branch: refs/heads/HDFS-7240
Commit: 13eda5000304099d1145631f9be13ce8a00b600d
Parents: d77ed23
Author: Steve Loughran <st...@apache.org>
Authored: Thu Sep 7 18:35:03 2017 +0100
Committer: Steve Loughran <st...@apache.org>
Committed: Thu Sep 7 18:35:03 2017 +0100

----------------------------------------------------------------------
 .../fs/azure/AzureNativeFileSystemStore.java    |   73 +-
 .../hadoop/fs/azure/BlockBlobAppendStream.java  | 1301 +++++++++++-------
 .../hadoop/fs/azure/NativeAzureFileSystem.java  |   77 +-
 .../hadoop/fs/azure/NativeFileSystemStore.java  |    5 +-
 .../fs/azure/SecureStorageInterfaceImpl.java    |   10 +-
 .../hadoop/fs/azure/SelfRenewingLease.java      |   10 +-
 .../hadoop/fs/azure/StorageInterface.java       |    3 +-
 .../hadoop/fs/azure/StorageInterfaceImpl.java   |   12 +-
 .../fs/azure/SyncableDataOutputStream.java      |   11 +
 .../hadoop-azure/src/site/markdown/index.md     |   34 +
 .../hadoop/fs/azure/MockStorageInterface.java   |    3 +-
 .../azure/TestAzureConcurrentOutOfBandIo.java   |    6 +-
 ...estNativeAzureFileSystemBlockCompaction.java |  266 ++++
 .../src/test/resources/log4j.properties         |    1 +
 14 files changed, 1273 insertions(+), 539 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/13eda500/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/AzureNativeFileSystemStore.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/AzureNativeFileSystemStore.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/AzureNativeFileSystemStore.java
index bd8ac68..639862f 100644
--- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/AzureNativeFileSystemStore.java
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/AzureNativeFileSystemStore.java
@@ -203,6 +203,23 @@ public class AzureNativeFileSystemStore implements NativeFileSystemStore {
   private Set<String> pageBlobDirs;
 
   /**
+   * Configuration key to indicate the set of directories in WASB where we
+   * should store files as block blobs with block compaction enabled.
+   *
+   * Entries can be directory paths relative to the container (e.g. "/path") or
+   * fully qualified wasb:// URIs (e.g.
+   * wasb://container@example.blob.core.windows.net/path)
+   */
+  public static final String KEY_BLOCK_BLOB_WITH_COMPACTION_DIRECTORIES =
+          "fs.azure.block.blob.with.compaction.dir";
+
+  /**
+   * The set of directories where we should store files as block blobs with
+   * block compaction enabled.
+   */
+  private Set<String> blockBlobWithCompationDirs;
+
+  /**
    * Configuration key to indicate the set of directories in WASB where
    * we should do atomic folder rename synchronized with createNonRecursive.
    */
@@ -527,6 +544,12 @@ public class AzureNativeFileSystemStore implements NativeFileSystemStore {
     // User-agent
     userAgentId = conf.get(USER_AGENT_ID_KEY, USER_AGENT_ID_DEFAULT);
 
+    // Extract the directories that should contain block blobs with compaction
+    blockBlobWithCompationDirs = getDirectorySet(
+        KEY_BLOCK_BLOB_WITH_COMPACTION_DIRECTORIES);
+    LOG.debug("Block blobs with compaction directories:  {}",
+        setToString(blockBlobWithCompationDirs));
+
     // Extract directories that should have atomic rename applied.
     atomicRenameDirs = getDirectorySet(KEY_ATOMIC_RENAME_DIRECTORIES);
     String hbaseRoot;
@@ -1165,6 +1188,17 @@ public class AzureNativeFileSystemStore implements NativeFileSystemStore {
   }
 
   /**
+   * Checks if the given key in Azure Storage should be stored as a block blobs
+   * with compaction enabled instead of normal block blob.
+   *
+   * @param key blob name
+   * @return true, if the file is in directory with block compaction enabled.
+   */
+  public boolean isBlockBlobWithCompactionKey(String key) {
+    return isKeyForDirectorySet(key, blockBlobWithCompationDirs);
+  }
+
+  /**
    * Checks if the given key in Azure storage should have synchronized
    * atomic folder rename createNonRecursive implemented.
    */
@@ -1356,7 +1390,9 @@ public class AzureNativeFileSystemStore implements NativeFileSystemStore {
   }
 
   @Override
-  public DataOutputStream storefile(String key, PermissionStatus permissionStatus)
+  public DataOutputStream storefile(String keyEncoded,
+                                    PermissionStatus permissionStatus,
+                                    String key)
       throws AzureException {
     try {
 
@@ -1417,12 +1453,26 @@ public class AzureNativeFileSystemStore implements NativeFileSystemStore {
 
       // Get the blob reference from the store's container and
       // return it.
-      CloudBlobWrapper blob = getBlobReference(key);
+      CloudBlobWrapper blob = getBlobReference(keyEncoded);
       storePermissionStatus(blob, permissionStatus);
 
       // Create the output stream for the Azure blob.
       //
-      OutputStream outputStream = openOutputStream(blob);
+      OutputStream outputStream;
+
+      if (isBlockBlobWithCompactionKey(key)) {
+        BlockBlobAppendStream blockBlobOutputStream = new BlockBlobAppendStream(
+            (CloudBlockBlobWrapper) blob,
+            keyEncoded,
+            this.uploadBlockSizeBytes,
+            true,
+            getInstrumentedContext());
+
+        outputStream = blockBlobOutputStream;
+      } else {
+        outputStream = openOutputStream(blob);
+      }
+
       DataOutputStream dataOutStream = new SyncableDataOutputStream(outputStream);
       return dataOutStream;
     } catch (Exception e) {
@@ -2869,10 +2919,21 @@ public class AzureNativeFileSystemStore implements NativeFileSystemStore {
 
       CloudBlobWrapper blob =  this.container.getBlockBlobReference(key);
 
-      BlockBlobAppendStream appendStream = new BlockBlobAppendStream((CloudBlockBlobWrapper) blob, key, bufferSize, getInstrumentedContext());
-      appendStream.initialize();
+      OutputStream outputStream;
+
+      BlockBlobAppendStream blockBlobOutputStream = new BlockBlobAppendStream(
+          (CloudBlockBlobWrapper) blob,
+          key,
+          bufferSize,
+          isBlockBlobWithCompactionKey(key),
+          getInstrumentedContext());
+
+      outputStream = blockBlobOutputStream;
+
+      DataOutputStream dataOutStream = new SyncableDataOutputStream(
+          outputStream);
 
-      return new DataOutputStream(appendStream);
+      return dataOutStream;
     } catch(Exception ex) {
       throw new AzureException(ex);
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/13eda500/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/BlockBlobAppendStream.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/BlockBlobAppendStream.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/BlockBlobAppendStream.java
index afb9379..84342cd 100644
--- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/BlockBlobAppendStream.java
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/BlockBlobAppendStream.java
@@ -22,122 +22,256 @@ import java.io.ByteArrayInputStream;
 import java.io.ByteArrayOutputStream;
 import java.io.IOException;
 import java.io.OutputStream;
+import java.nio.ByteBuffer;
 import java.nio.charset.StandardCharsets;
 import java.util.ArrayList;
-import java.util.Calendar;
-import java.util.HashMap;
-import java.util.Locale;
+import java.util.Iterator;
 import java.util.List;
 import java.util.UUID;
 import java.util.Random;
-import java.util.TimeZone;
+import java.util.concurrent.ConcurrentLinkedDeque;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.Semaphore;
 import java.util.concurrent.ThreadFactory;
 import java.util.concurrent.ThreadPoolExecutor;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.atomic.AtomicReference;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import org.apache.commons.lang.StringUtils;
 
 import org.apache.hadoop.fs.FSExceptionMessages;
 import org.apache.commons.codec.binary.Base64;
+import org.apache.hadoop.fs.StreamCapabilities;
+import org.apache.hadoop.fs.Syncable;
 import org.apache.hadoop.fs.azure.StorageInterface.CloudBlockBlobWrapper;
-import org.eclipse.jetty.util.log.Log;
+import org.apache.hadoop.io.ElasticByteBufferPool;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import com.microsoft.azure.storage.AccessCondition;
 import com.microsoft.azure.storage.OperationContext;
 import com.microsoft.azure.storage.StorageException;
+import com.microsoft.azure.storage.StorageErrorCodeStrings;
 import com.microsoft.azure.storage.blob.BlobRequestOptions;
 import com.microsoft.azure.storage.blob.BlockEntry;
 import com.microsoft.azure.storage.blob.BlockListingFilter;
+import com.microsoft.azure.storage.blob.BlockSearchMode;
+
+import static org.apache.hadoop.fs.StreamCapabilities.StreamCapability.HFLUSH;
+import static org.apache.hadoop.fs.StreamCapabilities.StreamCapability.HSYNC;
 
 /**
- * Stream object that implememnts append for Block Blobs in WASB.
+ * Stream object that implements append for Block Blobs in WASB.
+ *
+ * The stream object implements hflush/hsync and block compaction. Block
+ * compaction is the process of replacing a sequence of small blocks with one
+ * big block. Azure Block blobs supports up to 50000 blocks and every
+ * hflush/hsync generates one block. When the number of blocks is above 32000,
+ * the process of compaction decreases the total number of blocks, if possible.
+ * If compaction is disabled, hflush/hsync are empty functions.
+ *
+ * The stream object uses background threads for uploading the blocks and the
+ * block blob list. Blocks can be uploaded concurrently. However, when the block
+ * list is uploaded, block uploading should stop. If a block is uploaded before
+ * the block list and the block id is not in the list, the block will be lost.
+ * If the block is uploaded after the block list and the block id is in the
+ * list, the block list upload will fail. The exclusive access for the block
+ * list upload is managed by uploadingSemaphore.
  */
-public class BlockBlobAppendStream extends OutputStream {
 
+public class BlockBlobAppendStream extends OutputStream implements Syncable,
+    StreamCapabilities {
+
+  /**
+   * The name of the blob/file.
+   */
   private final String key;
-  private final int bufferSize;
-  private ByteArrayOutputStream outBuffer;
-  private final CloudBlockBlobWrapper blob;
-  private final OperationContext opContext;
 
   /**
-   * Variable to track if the stream has been closed.
+   * This variable tracks if this is new blob or existing one.
+   */
+  private boolean blobExist;
+
+  /**
+   * When the blob exist, to to prevent concurrent write we take a lease.
+   * Taking a lease is not necessary for new blobs.
    */
-  private boolean closed = false;
+  private SelfRenewingLease lease = null;
 
   /**
-   * Variable to track if the append lease is released.
+   * The support for process of compaction is optional.
    */
+  private final boolean compactionEnabled;
 
-  private volatile boolean leaseFreed;
+  /**
+   * The number of blocks above each block compaction is triggered.
+   */
+  private static final int DEFAULT_ACTIVATE_COMPACTION_BLOCK_COUNT = 32000;
 
   /**
-   * Variable to track if the append stream has been
-   * initialized.
+   * The number of blocks above each block compaction is triggered.
    */
+  private int activateCompactionBlockCount
+      = DEFAULT_ACTIVATE_COMPACTION_BLOCK_COUNT;
 
-  private boolean initialized = false;
+  /**
+   * The size of the output buffer. Writes store the data in outBuffer until
+   * either the size is above maxBlockSize or hflush/hsync is called.
+   */
+  private final AtomicInteger maxBlockSize;
 
   /**
-   *  Last IOException encountered
+   * The current buffer where writes are stored.
    */
-  private volatile IOException lastError = null;
+  private ByteBuffer outBuffer;
 
   /**
-   * List to keep track of the uncommitted azure storage
-   * block ids
+   * The size of the blob that has been successfully stored in the Azure Blob
+   * service.
    */
-  private final List<BlockEntry> uncommittedBlockEntries;
+  private final AtomicLong committedBlobLength = new AtomicLong(0);
 
-  private static final int UNSET_BLOCKS_COUNT = -1;
+  /**
+   * Position of last block in the blob.
+   */
+  private volatile long blobLength = 0;
 
   /**
-   * Variable to hold the next block id to be used for azure
-   * storage blocks.
+   * Minutes waiting before the close operation timed out.
    */
-  private long nextBlockCount = UNSET_BLOCKS_COUNT;
+  private static final int CLOSE_UPLOAD_DELAY = 10;
 
   /**
-   * Variable to hold the block id prefix to be used for azure
-   * storage blocks from azure-storage-java sdk version 4.2.0 onwards
+   * Keep alive time for the threadpool.
    */
-  private String blockIdPrefix = null;
+  private static final int THREADPOOL_KEEP_ALIVE = 30;
+  /**
+   * Azure Block Blob used for the stream.
+   */
+  private final CloudBlockBlobWrapper blob;
+
+  /**
+   * Azure Storage operation context.
+   */
+  private final OperationContext opContext;
+
+  /**
+   * Commands send from client calls to the background thread pool.
+   */
+  private abstract class UploadCommand {
+
+    // the blob offset for the command
+    private final long commandBlobOffset;
+
+    // command completion latch
+    private final CountDownLatch completed = new CountDownLatch(1);
+
+    UploadCommand(long offset) {
+      this.commandBlobOffset = offset;
+    }
+
+    long getCommandBlobOffset() {
+      return commandBlobOffset;
+    }
+
+    void await() throws InterruptedException {
+      completed.await();
+    }
+
+    void awaitAsDependent() throws InterruptedException {
+      await();
+    }
+
+    void setCompleted() {
+      completed.countDown();
+    }
 
-  private final Random sequenceGenerator = new Random();
+    void execute() throws InterruptedException, IOException {}
+
+    void dump() {}
+  }
+
+  /**
+   * The list of recent commands. Before block list is committed, all the block
+   * listed in the list must be uploaded. activeBlockCommands is used for
+   * enumerating the blocks and waiting on the latch until the block is
+   * uploaded.
+   */
+  private final ConcurrentLinkedQueue<UploadCommand> activeBlockCommands
+      = new ConcurrentLinkedQueue<>();
+
+  /**
+   * Variable to track if the stream has been closed.
+   */
+  private volatile boolean closed = false;
+
+  /**
+   *  First IOException encountered.
+   */
+  private final AtomicReference<IOException> firstError
+          = new AtomicReference<>();
+
+  /**
+   * Flag set when the first error has been thrown.
+   */
+  private boolean firstErrorThrown = false;
 
   /**
-   *  Time to wait to renew lease in milliseconds
+   * Semaphore for serializing block uploads with NativeAzureFileSystem.
+   *
+   * The semaphore starts with number of permits equal to the number of block
+   * upload threads. Each block upload thread needs one permit to start the
+   * upload. The put block list acquires all the permits before the block list
+   * is committed.
    */
-  private static final int LEASE_RENEWAL_PERIOD = 10000;
+  private final Semaphore uploadingSemaphore = new Semaphore(
+      MAX_NUMBER_THREADS_IN_THREAD_POOL,
+      true);
 
   /**
-   *  Number of times to retry for lease renewal
+   * Queue storing buffers with the size of the Azure block ready for
+   * reuse. The pool allows reusing the blocks instead of allocating new
+   * blocks. After the data is sent to the service, the buffer is returned
+   * back to the queue
    */
-  private static final int MAX_LEASE_RENEWAL_RETRY_COUNT = 3;
+  private final ElasticByteBufferPool poolReadyByteBuffers
+          = new ElasticByteBufferPool();
 
   /**
-   *  Time to wait before retrying to set the lease
+   * The blob's block list.
    */
-  private static final int LEASE_RENEWAL_RETRY_SLEEP_PERIOD = 500;
+  private final List<BlockEntry> blockEntries = new ArrayList<>(
+      DEFAULT_CAPACITY_BLOCK_ENTRIES);
+
+  private static final int DEFAULT_CAPACITY_BLOCK_ENTRIES = 1024;
 
   /**
-   *  Metadata key used on the blob to indicate append lease is active
+   * The uncommitted blob's block list.
    */
-  public static final String APPEND_LEASE = "append_lease";
+  private final ConcurrentLinkedDeque<BlockEntry> uncommittedBlockEntries
+      = new ConcurrentLinkedDeque<>();
+
+  /**
+   * Variable to hold the next block id to be used for azure storage blocks.
+   */
+  private static final int UNSET_BLOCKS_COUNT = -1;
+  private long nextBlockCount = UNSET_BLOCKS_COUNT;
 
   /**
-   * Timeout value for the append lease in millisecs. If the lease is not
-   * renewed within 30 seconds then another thread can acquire the append lease
-   * on the blob
+   * Variable to hold the block id prefix to be used for azure storage blocks.
    */
-  public static final int APPEND_LEASE_TIMEOUT = 30000;
+  private String blockIdPrefix = null;
 
   /**
-   *  Metdata key used on the blob to indicate last modified time of append lease
+   *  Maximum number of threads in block upload thread pool.
    */
-  public static final String APPEND_LEASE_LAST_MODIFIED = "append_lease_last_modified";
+  private static final int MAX_NUMBER_THREADS_IN_THREAD_POOL = 4;
 
   /**
    * Number of times block upload needs is retried.
@@ -145,17 +279,33 @@ public class BlockBlobAppendStream extends OutputStream {
   private static final int MAX_BLOCK_UPLOAD_RETRIES = 3;
 
   /**
-   * Wait time between block upload retries in millisecs.
+   * Wait time between block upload retries in milliseconds.
    */
   private static final int BLOCK_UPLOAD_RETRY_INTERVAL = 1000;
 
-  private static final Logger LOG = LoggerFactory.getLogger(BlockBlobAppendStream.class);
+  /**
+   * Logger.
+   */
+  private static final Logger LOG =
+          LoggerFactory.getLogger(BlockBlobAppendStream.class);
 
+  /**
+   * The absolute maximum of blocks for a blob. It includes committed and
+   * temporary blocks.
+   */
   private static final int MAX_BLOCK_COUNT = 100000;
 
+  /**
+   * The upload thread pool executor.
+   */
   private ThreadPoolExecutor ioThreadPool;
 
   /**
+   * Azure Storage access conditions for the blob.
+   */
+  private final AccessCondition accessCondition = new AccessCondition();
+
+  /**
    * Atomic integer to provide thread id for thread names for uploader threads.
    */
   private final AtomicInteger threadSequenceNumber;
@@ -163,106 +313,123 @@ public class BlockBlobAppendStream extends OutputStream {
   /**
    * Prefix to be used for thread names for uploader threads.
    */
-  private static final String THREAD_ID_PREFIX = "BlockBlobAppendStream";
-
-  private static final String UTC_STR = "UTC";
+  private static final String THREAD_ID_PREFIX = "append-blockblob";
 
+  /**
+   * BlockBlobAppendStream constructor.
+   *
+   * @param blob
+   *          Azure Block Blob
+   * @param aKey
+   *          blob's name
+   * @param bufferSize
+   *          the maximum size of a blob block.
+   * @param compactionEnabled
+   *          is the compaction process enabled for this blob
+   * @param opContext
+   *          Azure Store operation context for the blob
+   * @throws IOException
+   *           if an I/O error occurs. In particular, an IOException may be
+   *           thrown if the output stream cannot be used for append operations
+   */
   public BlockBlobAppendStream(final CloudBlockBlobWrapper blob,
-      final String aKey, final int bufferSize, final OperationContext opContext)
+                               final String aKey,
+                               final int bufferSize,
+                               final boolean compactionEnabled,
+                               final OperationContext opContext)
           throws IOException {
 
-    if (null == aKey || 0 == aKey.length()) {
-      throw new IllegalArgumentException(
-          "Illegal argument: The key string is null or empty");
-    }
-
-    if (0 >= bufferSize) {
-      throw new IllegalArgumentException(
-          "Illegal argument bufferSize cannot be zero or negative");
-    }
-
+    Preconditions.checkArgument(StringUtils.isNotEmpty(aKey));
+    Preconditions.checkArgument(bufferSize >= 0);
 
     this.blob = blob;
     this.opContext = opContext;
     this.key = aKey;
-    this.bufferSize = bufferSize;
+    this.maxBlockSize = new AtomicInteger(bufferSize);
     this.threadSequenceNumber = new AtomicInteger(0);
     this.blockIdPrefix = null;
-    setBlocksCountAndBlockIdPrefix();
+    this.compactionEnabled = compactionEnabled;
+    this.blobExist = true;
+    this.outBuffer = poolReadyByteBuffers.getBuffer(false, maxBlockSize.get());
 
-    this.outBuffer = new ByteArrayOutputStream(bufferSize);
-    this.uncommittedBlockEntries = new ArrayList<BlockEntry>();
-
-    // Acquire append lease on the blob.
     try {
-      //Set the append lease if the value of the append lease is false
-      if (!updateBlobAppendMetadata(true, false)) {
-        LOG.error("Unable to set Append Lease on the Blob : {} "
-            + "Possibly because another client already has a create or append stream open on the Blob", key);
-        throw new IOException("Unable to set Append lease on the Blob. "
-            + "Possibly because another client already had an append stream open on the Blob.");
-      }
-    } catch (StorageException ex) {
-      LOG.error("Encountered Storage exception while acquiring append "
-          + "lease on blob : {}. Storage Exception : {} ErrorCode : {}",
-          key, ex, ex.getErrorCode());
+      // download the block list
+      blockEntries.addAll(
+          blob.downloadBlockList(
+              BlockListingFilter.COMMITTED,
+              new BlobRequestOptions(),
+              opContext));
+
+      blobLength = blob.getProperties().getLength();
+
+      committedBlobLength.set(blobLength);
 
-      throw new IOException(ex);
+      // Acquiring lease on the blob.
+      lease = new SelfRenewingLease(blob, true);
+      accessCondition.setLeaseID(lease.getLeaseID());
+
+    } catch (StorageException ex) {
+      if (ex.getErrorCode().equals(StorageErrorCodeStrings.BLOB_NOT_FOUND)) {
+        blobExist = false;
+      }
+      else if (ex.getErrorCode().equals(
+              StorageErrorCodeStrings.LEASE_ALREADY_PRESENT)) {
+        throw new AzureException(
+                "Unable to set Append lease on the Blob: " + ex, ex);
+      }
+      else {
+        LOG.debug(
+            "Encountered storage exception."
+                + " StorageException : {} ErrorCode : {}",
+            ex,
+            ex.getErrorCode());
+        throw new AzureException(ex);
+      }
     }
 
-    leaseFreed = false;
+    setBlocksCountAndBlockIdPrefix(blockEntries);
+
+    this.ioThreadPool = new ThreadPoolExecutor(
+        MAX_NUMBER_THREADS_IN_THREAD_POOL,
+        MAX_NUMBER_THREADS_IN_THREAD_POOL,
+        THREADPOOL_KEEP_ALIVE,
+        TimeUnit.SECONDS,
+        new LinkedBlockingQueue<>(),
+        new UploaderThreadFactory());
   }
 
   /**
-   * Helper method that starts an Append Lease renewer thread and the
-   * thread pool.
+   * Set payload size of the stream.
+   * It is intended to be used for unit testing purposes only.
    */
-  public synchronized void initialize() {
-
-    if (initialized) {
-      return;
-    }
-    /*
-     * Start the thread for  Append lease renewer.
-     */
-    Thread appendLeaseRenewer = new Thread(new AppendRenewer());
-    appendLeaseRenewer.setDaemon(true);
-    appendLeaseRenewer.setName(String.format("%s-AppendLeaseRenewer", key));
-    appendLeaseRenewer.start();
-
-    /*
-     * Parameters to ThreadPoolExecutor:
-     * corePoolSize : the number of threads to keep in the pool, even if they are idle,
-     *                unless allowCoreThreadTimeOut is set
-     * maximumPoolSize : the maximum number of threads to allow in the pool
-     * keepAliveTime - when the number of threads is greater than the core,
-     *                 this is the maximum time that excess idle threads will
-     *                 wait for new tasks before terminating.
-     * unit - the time unit for the keepAliveTime argument
-     * workQueue - the queue to use for holding tasks before they are executed
-     *  This queue will hold only the Runnable tasks submitted by the execute method.
-     */
-    this.ioThreadPool = new ThreadPoolExecutor(4, 4, 2, TimeUnit.SECONDS,
-        new LinkedBlockingQueue<Runnable>(), new UploaderThreadFactory());
+  @VisibleForTesting
+  synchronized void setMaxBlockSize(int size) {
+    maxBlockSize.set(size);
 
-    initialized = true;
+    // it is for testing only so we can abandon the previously allocated
+    // payload
+    this.outBuffer = ByteBuffer.allocate(maxBlockSize.get());
   }
 
   /**
-   * Get the blob name.
-   *
-   * @return String Blob name.
+   * Set compaction parameters.
+   * It is intended to be used for unit testing purposes only.
    */
-  public String getKey() {
-    return key;
+  @VisibleForTesting
+  void setCompactionBlockCount(int activationCount) {
+    activateCompactionBlockCount = activationCount;
   }
 
   /**
-   * Get the backing blob.
-   * @return buffer size of the stream.
+   * Get the list of block entries. It is used for testing purposes only.
+   * @return List of block entries.
    */
-  public int getBufferSize() {
-    return bufferSize;
+  @VisibleForTesting
+  List<BlockEntry> getBlockList() throws StorageException, IOException {
+    return blob.downloadBlockList(
+        BlockListingFilter.COMMITTED,
+        new BlobRequestOptions(),
+        opContext);
   }
 
   /**
@@ -283,21 +450,6 @@ public class BlockBlobAppendStream extends OutputStream {
   }
 
   /**
-   * Writes b.length bytes from the specified byte array to this output stream.
-   *
-   * @param data
-   *          the byte array to write.
-   *
-   * @throws IOException
-   *           if an I/O error occurs. In particular, an IOException may be
-   *           thrown if the output stream has been closed.
-   */
-  @Override
-  public void write(final byte[] data) throws IOException {
-    write(data, 0, data.length);
-  }
-
-  /**
    * Writes length bytes from the specified byte array starting at offset to
    * this output stream.
    *
@@ -312,529 +464,678 @@ public class BlockBlobAppendStream extends OutputStream {
    *           thrown if the output stream has been closed.
    */
   @Override
-  public void write(final byte[] data, final int offset, final int length)
+  public synchronized void write(final byte[] data, int offset, int length)
       throws IOException {
+    Preconditions.checkArgument(data != null, "null data");
 
     if (offset < 0 || length < 0 || length > data.length - offset) {
-      throw new IndexOutOfBoundsException("write API in append stream called with invalid arguments");
-    }
-
-    writeInternal(data, offset, length);
-  }
-
-  @Override
-  public synchronized void close() throws IOException {
-
-    if (!initialized) {
-      throw new IOException("Trying to close an uninitialized Append stream");
+      throw new IndexOutOfBoundsException();
     }
 
     if (closed) {
-      return;
+      throw new IOException(FSExceptionMessages.STREAM_IS_CLOSED);
     }
 
-    if (leaseFreed) {
-      throw new IOException(String.format("Attempting to close an append stream on blob : %s "
-          + " that does not have lease on the Blob. Failing close", key));
-    }
+    while (outBuffer.remaining() < length) {
+
+      int remaining = outBuffer.remaining();
+      outBuffer.put(data, offset, remaining);
+
+      // upload payload to azure storage
+      addBlockUploadCommand();
 
-    if (outBuffer.size() > 0) {
-      uploadBlockToStorage(outBuffer.toByteArray());
+      offset += remaining;
+      length -= remaining;
     }
 
-    ioThreadPool.shutdown();
+    outBuffer.put(data, offset, length);
+  }
 
-    try {
-      if (!ioThreadPool.awaitTermination(10, TimeUnit.MINUTES)) {
-        LOG.error("Time out occurred while waiting for IO request to finish in append"
-            + " for blob : {}", key);
-        NativeAzureFileSystemHelper.logAllLiveStackTraces();
-        throw new IOException("Timed out waiting for IO requests to finish");
-      }
-    } catch(InterruptedException intrEx) {
 
-      // Restore the interrupted status
-      Thread.currentThread().interrupt();
-      LOG.error("Upload block operation in append interrupted for blob {}. Failing close", key);
-      throw new IOException("Append Commit interrupted.");
-    }
+  /**
+   * Flushes this output stream and forces any buffered output bytes to be
+   * written out. If any data remains in the payload it is committed to the
+   * service. Data is queued for writing and forced out to the service
+   * before the call returns.
+   */
+  @Override
+  public void flush() throws IOException {
 
-    // Calling commit after all blocks are succesfully uploaded.
-    if (lastError == null) {
-      commitAppendBlocks();
+    if (closed) {
+      // calling close() after the stream is closed starts with call to flush()
+      return;
     }
 
-    // Perform cleanup.
-    cleanup();
+    addBlockUploadCommand();
 
-    if (lastError != null) {
-      throw lastError;
+    if (committedBlobLength.get() < blobLength) {
+      try {
+        // wait until the block list is committed
+        addFlushCommand().await();
+      } catch (InterruptedException ie) {
+        Thread.currentThread().interrupt();
+      }
     }
   }
 
   /**
-   * Helper method that cleans up the append stream.
+   * Force all data in the output stream to be written to Azure storage.
+   * Wait to return until this is complete.
    */
-  private synchronized void cleanup() {
-
-    closed = true;
-
-    try {
-      // Set the value of append lease to false if the value is set to true.
-        updateBlobAppendMetadata(false, true);
-    } catch(StorageException ex) {
-      LOG.debug("Append metadata update on the Blob : {} encountered Storage Exception : {} "
-          + "Error Code : {}",
-          key, ex, ex.getErrorCode());
-      lastError = new IOException(ex);
+  @Override
+  public void hsync() throws IOException {
+    // when block compaction is disabled, hsync is empty function
+    if (compactionEnabled) {
+      flush();
     }
+  }
 
-    leaseFreed = true;
+  /**
+   * Force all data in the output stream to be written to Azure storage.
+   * Wait to return until this is complete.
+   */
+  @Override
+  public void hflush() throws IOException {
+    // when block compaction is disabled, hflush is empty function
+    if (compactionEnabled) {
+      flush();
+    }
   }
 
   /**
-   * Method to commit all the uncommited blocks to azure storage.
-   * If the commit fails then blocks are automatically cleaned up
-   * by Azure storage.
-   * @throws IOException
+   * The Synchronization capabilities of this stream depend upon the compaction
+   * policy.
+   * @param capability string to query the stream support for.
+   * @return true for hsync and hflush when compaction is enabled.
    */
-  private synchronized void commitAppendBlocks() throws IOException {
+  @Override
+  public boolean hasCapability(String capability) {
+    return compactionEnabled
+        && (capability.equalsIgnoreCase(HSYNC.getValue())
+        || capability.equalsIgnoreCase((HFLUSH.getValue())));
+  }
 
-    SelfRenewingLease lease = null;
+  /**
+   * Force all data in the output stream to be written to Azure storage.
+   * Wait to return until this is complete. Close the access to the stream and
+   * shutdown the upload thread pool.
+   * If the blob was created, its lease will be released.
+   * Any error encountered caught in threads and stored will be rethrown here
+   * after cleanup.
+   */
+  @Override
+  public synchronized void close() throws IOException {
 
-    try {
-      if (uncommittedBlockEntries.size() > 0) {
+    LOG.debug("close {} ", key);
 
-        //Acquiring lease on the blob.
-        lease = new SelfRenewingLease(blob);
+    if (closed) {
+      return;
+    }
 
-        // Downloading existing blocks
-        List<BlockEntry> blockEntries =  blob.downloadBlockList(BlockListingFilter.COMMITTED,
-            new BlobRequestOptions(), opContext);
+    // Upload the last block regardless of compactionEnabled flag
+    flush();
 
-        // Adding uncommitted blocks.
-        blockEntries.addAll(uncommittedBlockEntries);
+    // Initiates an orderly shutdown in which previously submitted tasks are
+    // executed.
+    ioThreadPool.shutdown();
 
-        AccessCondition accessCondition = new AccessCondition();
-        accessCondition.setLeaseID(lease.getLeaseID());
-        blob.commitBlockList(blockEntries, accessCondition, new BlobRequestOptions(), opContext);
-        uncommittedBlockEntries.clear();
+    try {
+      // wait up to CLOSE_UPLOAD_DELAY minutes to upload all the blocks
+      if (!ioThreadPool.awaitTermination(CLOSE_UPLOAD_DELAY, TimeUnit.MINUTES)) {
+        LOG.error("Time out occurred while close() is waiting for IO request to"
+            + " finish in append"
+            + " for blob : {}",
+            key);
+        NativeAzureFileSystemHelper.logAllLiveStackTraces();
+        throw new AzureException("Timed out waiting for IO requests to finish");
       }
-    } catch(StorageException ex) {
-      LOG.error("Storage exception encountered during block commit phase of append for blob"
-          + " : {} Storage Exception : {} Error Code: {}", key, ex, ex.getErrorCode());
-      throw new IOException("Encountered Exception while committing append blocks", ex);
-    } finally {
-      if (lease != null) {
+    } catch(InterruptedException ex) {
+      Thread.currentThread().interrupt();
+    }
+
+    // release the lease
+    if (firstError.get() == null && blobExist) {
         try {
           lease.free();
-        } catch(StorageException ex) {
-          LOG.debug("Exception encountered while releasing lease for "
-              + "blob : {} StorageException : {} ErrorCode : {}", key, ex, ex.getErrorCode());
-          // Swallowing exception here as the lease is cleaned up by the SelfRenewingLease object.
+        } catch (StorageException ex) {
+          LOG.debug("Lease free update blob {} encountered Storage Exception:"
+              + " {} Error Code : {}",
+              key,
+              ex,
+              ex.getErrorCode());
+          maybeSetFirstError(new AzureException(ex));
         }
-      }
+    }
+
+    closed = true;
+
+    // finally, throw the first exception raised if it has not
+    // been thrown elsewhere.
+    if (firstError.get() != null && !firstErrorThrown) {
+      throw firstError.get();
     }
   }
 
   /**
-   * Helper method used to generate the blockIDs. The algorithm used is similar to the Azure
-   * storage SDK.
+   * Helper method used to generate the blockIDs. The algorithm used is similar
+   * to the Azure storage SDK.
    */
-  private void setBlocksCountAndBlockIdPrefix() throws IOException {
+  private void setBlocksCountAndBlockIdPrefix(List<BlockEntry> blockEntries) {
 
-    try {
+    if (nextBlockCount == UNSET_BLOCKS_COUNT && blockIdPrefix == null) {
 
-      if (nextBlockCount == UNSET_BLOCKS_COUNT && blockIdPrefix==null) {
+      Random sequenceGenerator = new Random();
 
-        List<BlockEntry> blockEntries =
-            blob.downloadBlockList(BlockListingFilter.COMMITTED, new BlobRequestOptions(), opContext);
+      String blockZeroBlockId = (!blockEntries.isEmpty())
+          ? blockEntries.get(0).getId()
+          : "";
+      String prefix = UUID.randomUUID().toString() + "-";
+      String sampleNewerVersionBlockId = generateNewerVersionBlockId(prefix,
+          0);
 
-        String blockZeroBlockId = (blockEntries.size() > 0) ? blockEntries.get(0).getId() : "";
-        String prefix = UUID.randomUUID().toString() + "-";
-        String sampleNewerVersionBlockId = generateNewerVersionBlockId(prefix, 0);
+      if (!blockEntries.isEmpty()
+          && blockZeroBlockId.length() < sampleNewerVersionBlockId.length()) {
 
-        if (blockEntries.size() > 0 && blockZeroBlockId.length() < sampleNewerVersionBlockId.length()) {
+        // If blob has already been created with 2.2.0, append subsequent blocks
+        // with older version (2.2.0) blockId compute nextBlockCount, the way it
+        // was done before; and don't use blockIdPrefix
+        this.blockIdPrefix = "";
+        nextBlockCount = (long) (sequenceGenerator.nextInt(Integer.MAX_VALUE))
+            + sequenceGenerator.nextInt(
+                Integer.MAX_VALUE - MAX_BLOCK_COUNT);
+        nextBlockCount += blockEntries.size();
 
-          // If blob has already been created with 2.2.0, append subsequent blocks with older version (2.2.0) blockId
-          // compute nextBlockCount, the way it was done before; and don't use blockIdPrefix
-          this.blockIdPrefix = "";
-          nextBlockCount = (long) (sequenceGenerator.nextInt(Integer.MAX_VALUE))
-              + sequenceGenerator.nextInt(Integer.MAX_VALUE - MAX_BLOCK_COUNT);
-          nextBlockCount += blockEntries.size();
-
-        } else {
-
-          // If there are no existing blocks, create the first block with newer version (4.2.0) blockId
-          // If blob has already been created with 4.2.0, append subsequent blocks with newer version (4.2.0) blockId
-          this.blockIdPrefix = prefix;
-          nextBlockCount = blockEntries.size();
-
-        }
+      } else {
 
+        // If there are no existing blocks, create the first block with newer
+        // version (4.2.0) blockId. If blob has already been created with 4.2.0,
+        // append subsequent blocks with newer version (4.2.0) blockId
+        this.blockIdPrefix = prefix;
+        nextBlockCount = blockEntries.size();
       }
-
-    } catch (StorageException ex) {
-      LOG.debug("Encountered storage exception during setting next Block Count and BlockId prefix."
-          + " StorageException : {} ErrorCode : {}", ex, ex.getErrorCode());
-      throw new IOException(ex);
     }
   }
 
   /**
-   * Helper method that generates the next block id for uploading a block to azure storage.
+   * Helper method that generates the next block id for uploading a block to
+   * azure storage.
    * @return String representing the block ID generated.
-   * @throws IOException
+   * @throws IOException if the stream is in invalid state
    */
   private String generateBlockId() throws IOException {
 
-    if (nextBlockCount == UNSET_BLOCKS_COUNT) {
-      throw new IOException("Append Stream in invalid state. nextBlockCount not set correctly");
-    }
-
-    if (this.blockIdPrefix == null) {
-      throw new IOException("Append Stream in invalid state. blockIdPrefix not set correctly");
-    }
-
-    if (!this.blockIdPrefix.equals("")) {
-
-      return generateNewerVersionBlockId(this.blockIdPrefix, nextBlockCount++);
-
-    } else {
-
-      return generateOlderVersionBlockId(nextBlockCount++);
-
+    if (nextBlockCount == UNSET_BLOCKS_COUNT || blockIdPrefix == null) {
+      throw new AzureException(
+            "Append Stream in invalid state. nextBlockCount not set correctly");
     }
 
+    return (!blockIdPrefix.isEmpty())
+        ? generateNewerVersionBlockId(blockIdPrefix, nextBlockCount++)
+        : generateOlderVersionBlockId(nextBlockCount++);
   }
 
   /**
-   * Helper method that generates an older (2.2.0) version blockId
+   * Helper method that generates an older (2.2.0) version blockId.
    * @return String representing the block ID generated.
    */
   private String generateOlderVersionBlockId(long id) {
 
-    byte[] blockIdInBytes = getBytesFromLong(id);
-    return new String(Base64.encodeBase64(blockIdInBytes), StandardCharsets.UTF_8);
+    byte[] blockIdInBytes = new byte[8];
+    for (int m = 0; m < 8; m++) {
+      blockIdInBytes[7 - m] = (byte) ((id >> (8 * m)) & 0xFF);
+    }
+
+    return new String(
+            Base64.encodeBase64(blockIdInBytes),
+            StandardCharsets.UTF_8);
   }
 
   /**
-   * Helper method that generates an newer (4.2.0) version blockId
+   * Helper method that generates an newer (4.2.0) version blockId.
    * @return String representing the block ID generated.
    */
   private String generateNewerVersionBlockId(String prefix, long id) {
 
     String blockIdSuffix  = String.format("%06d", id);
-    byte[] blockIdInBytes = (prefix + blockIdSuffix).getBytes(StandardCharsets.UTF_8);
+    byte[] blockIdInBytes =
+            (prefix + blockIdSuffix).getBytes(StandardCharsets.UTF_8);
     return new String(Base64.encodeBase64(blockIdInBytes), StandardCharsets.UTF_8);
   }
 
   /**
-   * Returns a byte array that represents the data of a <code>long</code> value. This
-   * utility method is copied from com.microsoft.azure.storage.core.Utility class.
-   * This class is marked as internal, hence we clone the method here and not express
-   * dependency on the Utility Class
-   *
-   * @param value
-   *            The value from which the byte array will be returned.
-   *
-   * @return A byte array that represents the data of the specified <code>long</code> value.
+   * This is shared between upload block Runnable and CommitBlockList. The
+   * method captures retry logic
+   * @param blockId block name
+   * @param dataPayload block content
    */
-  private static byte[] getBytesFromLong(final long value) {
+  private void writeBlockRequestInternal(String blockId,
+                                         ByteBuffer dataPayload,
+                                         boolean bufferPoolBuffer) {
+    IOException lastLocalException = null;
+
+    int uploadRetryAttempts = 0;
+    while (uploadRetryAttempts < MAX_BLOCK_UPLOAD_RETRIES) {
+      try {
+        long startTime = System.nanoTime();
+
+        blob.uploadBlock(blockId, accessCondition, new ByteArrayInputStream(
+            dataPayload.array()), dataPayload.position(),
+            new BlobRequestOptions(), opContext);
 
-    final byte[] tempArray = new byte[8];
+        LOG.debug("upload block finished for {} ms. block {} ",
+            TimeUnit.NANOSECONDS.toMillis(
+                System.nanoTime() - startTime), blockId);
+        break;
+
+      } catch(Exception ioe) {
+        LOG.debug("Encountered exception during uploading block for Blob {}"
+            + " Exception : {}", key, ioe);
+        uploadRetryAttempts++;
+        lastLocalException = new AzureException(
+            "Encountered Exception while uploading block: " + ioe, ioe);
+        try {
+          Thread.sleep(
+              BLOCK_UPLOAD_RETRY_INTERVAL * (uploadRetryAttempts + 1));
+        } catch(InterruptedException ie) {
+          Thread.currentThread().interrupt();
+          break;
+        }
+      }
+    }
 
-    for (int m = 0; m < 8; m++) {
-      tempArray[7 - m] = (byte) ((value >> (8 * m)) & 0xFF);
+    if (bufferPoolBuffer) {
+      poolReadyByteBuffers.putBuffer(dataPayload);
     }
 
-    return tempArray;
+    if (uploadRetryAttempts == MAX_BLOCK_UPLOAD_RETRIES) {
+      maybeSetFirstError(lastLocalException);
+    }
   }
 
   /**
-   * Helper method that creates a thread to upload a block to azure storage.
-   * @param payload
-   * @throws IOException
+   * Set {@link #firstError} to the exception if it is not already set.
+   * @param exception exception to save
    */
-  private synchronized void uploadBlockToStorage(byte[] payload)
-      throws IOException {
-
-    // upload payload to azure storage
-    String blockId = generateBlockId();
-
-    // Since uploads of the Azure storage are done in parallel threads, we go ahead
-    // add the blockId in the uncommitted list. If the upload of the block fails
-    // we don't commit the blockIds.
-    BlockEntry blockEntry = new BlockEntry(blockId);
-    blockEntry.setSize(payload.length);
-    uncommittedBlockEntries.add(blockEntry);
-    ioThreadPool.execute(new WriteRequest(payload, blockId));
+  private void maybeSetFirstError(IOException exception) {
+    firstError.compareAndSet(null, exception);
   }
 
 
   /**
-   * Helper method to updated the Blob metadata during Append lease operations.
-   * Blob metadata is updated to holdLease value only if the current lease
-   * status is equal to testCondition and the last update on the blob metadata
-   * is less that 30 secs old.
-   * @param holdLease
-   * @param testCondition
-   * @return true if the updated lease operation was successful or false otherwise
-   * @throws StorageException
+   * Throw the first error caught if it has not been raised already
+   * @throws IOException if one is caught and needs to be thrown.
    */
-  private boolean updateBlobAppendMetadata(boolean holdLease, boolean testCondition)
-      throws StorageException {
-
-    SelfRenewingLease lease = null;
-    StorageException lastStorageException = null;
-    int leaseRenewalRetryCount = 0;
-
-    /*
-     * Updating the Blob metadata honours following algorithm based on
-     *  1) If the append lease metadata is present
-     *  2) Last updated time of the append lease
-     *  3) Previous value of the Append lease metadata.
-     *
-     * The algorithm:
-     *  1) If append lease metadata is not part of the Blob. In this case
-     *     this is the first client to Append so we update the metadata.
-     *  2) If append lease metadata is present and timeout has occurred.
-     *     In this case irrespective of what the value of the append lease is we update the metadata.
-     *  3) If append lease metadata is present and is equal to testCondition value (passed as parameter)
-     *     and timeout has not occurred, we update the metadata.
-     *  4) If append lease metadata is present and is not equal to testCondition value (passed as parameter)
-     *     and timeout has not occurred, we do not update metadata and return false.
-     *
-     */
-    while (leaseRenewalRetryCount < MAX_LEASE_RENEWAL_RETRY_COUNT) {
-
-      lastStorageException = null;
-
-      synchronized(this) {
-        try {
-
-          final Calendar currentCalendar = Calendar
-              .getInstance(Locale.US);
-          currentCalendar.setTimeZone(TimeZone.getTimeZone(UTC_STR));
-          long currentTime = currentCalendar.getTime().getTime();
+  private void maybeThrowFirstError() throws IOException {
+    if (firstError.get() != null) {
+      firstErrorThrown = true;
+      throw firstError.get();
+    }
+  }
 
-          // Acquire lease on the blob.
-          lease = new SelfRenewingLease(blob);
+  /**
+   * Write block list. The method captures retry logic
+   */
+  private void writeBlockListRequestInternal() {
 
-          blob.downloadAttributes(opContext);
-          HashMap<String, String> metadata = blob.getMetadata();
+    IOException lastLocalException = null;
 
-          if (metadata.containsKey(APPEND_LEASE)
-              && currentTime - Long.parseLong(
-                  metadata.get(APPEND_LEASE_LAST_MODIFIED)) <= BlockBlobAppendStream.APPEND_LEASE_TIMEOUT
-              && !metadata.get(APPEND_LEASE).equals(Boolean.toString(testCondition))) {
-            return false;
-          }
+    int uploadRetryAttempts = 0;
+    while (uploadRetryAttempts < MAX_BLOCK_UPLOAD_RETRIES) {
+      try {
 
-          metadata.put(APPEND_LEASE, Boolean.toString(holdLease));
-          metadata.put(APPEND_LEASE_LAST_MODIFIED, Long.toString(currentTime));
-          blob.setMetadata(metadata);
-          AccessCondition accessCondition = new AccessCondition();
-          accessCondition.setLeaseID(lease.getLeaseID());
-          blob.uploadMetadata(accessCondition, null, opContext);
-          return true;
+        long startTime = System.nanoTime();
 
-        } catch (StorageException ex) {
-
-          lastStorageException = ex;
-          LOG.debug("Lease renewal for Blob : {} encountered Storage Exception : {} "
-              + "Error Code : {}",
-              key, ex, ex.getErrorCode());
-          leaseRenewalRetryCount++;
-
-        } finally {
-
-          if (lease != null) {
-            try {
-              lease.free();
-            } catch(StorageException ex) {
-              LOG.debug("Encountered Storage exception while releasing lease for Blob {} "
-                  + "during Append  metadata operation. Storage Exception {} "
-                  + "Error Code : {} ", key, ex, ex.getErrorCode());
-            } finally {
-              lease = null;
-            }
-          }
-        }
-      }
+        blob.commitBlockList(blockEntries, accessCondition,
+            new BlobRequestOptions(), opContext);
 
-      if (leaseRenewalRetryCount == MAX_LEASE_RENEWAL_RETRY_COUNT) {
-        throw lastStorageException;
-      } else {
+        LOG.debug("Upload block list took {} ms for blob {} ",
+                TimeUnit.NANOSECONDS.toMillis(
+                    System.nanoTime() - startTime), key);
+        break;
+
+      } catch(Exception ioe) {
+        LOG.debug("Encountered exception during uploading block for Blob {}"
+            + " Exception : {}", key, ioe);
+        uploadRetryAttempts++;
+        lastLocalException = new AzureException(
+            "Encountered Exception while uploading block: " + ioe, ioe);
         try {
-          Thread.sleep(LEASE_RENEWAL_RETRY_SLEEP_PERIOD);
-        } catch(InterruptedException ex) {
-          LOG.debug("Blob append metadata updated method interrupted");
+          Thread.sleep(
+              BLOCK_UPLOAD_RETRY_INTERVAL * (uploadRetryAttempts + 1));
+        } catch(InterruptedException ie) {
           Thread.currentThread().interrupt();
+          break;
         }
       }
     }
 
-    // The code should not enter here because the while loop will
-    // always be executed and if the while loop is executed we
-    // would returning from the while loop.
-    return false;
+    if (uploadRetryAttempts == MAX_BLOCK_UPLOAD_RETRIES) {
+      maybeSetFirstError(lastLocalException);
+    }
   }
 
   /**
-   * This is the only method that should be writing to outBuffer to maintain consistency of the outBuffer.
-   * @param data
-   * @param offset
-   * @param length
-   * @throws IOException
+   * A ThreadFactory that creates uploader thread with
+   * meaningful names helpful for debugging purposes.
    */
-  private synchronized void writeInternal(final byte[] data, final int offset, final int length)
-      throws IOException {
+  class UploaderThreadFactory implements ThreadFactory {
 
-    if (!initialized) {
-      throw new IOException("Trying to write to an un-initialized Append stream");
+    @Override
+    public Thread newThread(Runnable r) {
+      Thread t = new Thread(r);
+      t.setName(String.format("%s-%d", THREAD_ID_PREFIX,
+          threadSequenceNumber.getAndIncrement()));
+      return t;
     }
+  }
 
-    if (closed) {
-      throw new IOException(FSExceptionMessages.STREAM_IS_CLOSED);
-    }
+  /**
+   * Upload block commands.
+   */
+  private class UploadBlockCommand extends UploadCommand {
 
-    if (leaseFreed) {
-      throw new IOException(String.format("Write called on a append stream not holding lease. Failing Write"));
-    }
+    // the block content for upload
+    private final ByteBuffer payload;
 
-    byte[] currentData = new byte[length];
-    System.arraycopy(data, offset, currentData, 0, length);
+    // description of the block
+    private final BlockEntry entry;
 
-    // check to see if the data to be appended exceeds the
-    // buffer size. If so we upload a block to azure storage.
-    while ((outBuffer.size() + currentData.length) > bufferSize) {
+    UploadBlockCommand(String blockId, ByteBuffer payload) {
 
-      byte[] payload = new byte[bufferSize];
+      super(blobLength);
 
-      // Add data from the existing buffer
-      System.arraycopy(outBuffer.toByteArray(), 0, payload, 0, outBuffer.size());
+      BlockEntry blockEntry = new BlockEntry(blockId);
+      blockEntry.setSize(payload.position());
+      blockEntry.setSearchMode(BlockSearchMode.LATEST);
 
-      // Updating the available size in the payload
-      int availableSpaceInPayload = bufferSize - outBuffer.size();
+      this.payload = payload;
+      this.entry = blockEntry;
 
-      // Adding data from the current call
-      System.arraycopy(currentData, 0, payload, outBuffer.size(), availableSpaceInPayload);
+      uncommittedBlockEntries.add(blockEntry);
+    }
 
-      uploadBlockToStorage(payload);
+    /**
+     * Execute command.
+     */
+    void execute() throws InterruptedException {
+
+      uploadingSemaphore.acquire(1);
+      writeBlockRequestInternal(entry.getId(), payload, true);
+      uploadingSemaphore.release(1);
 
-      // updating the currentData buffer
-      byte[] tempBuffer = new byte[currentData.length - availableSpaceInPayload];
-      System.arraycopy(currentData, availableSpaceInPayload,
-          tempBuffer, 0, currentData.length - availableSpaceInPayload);
-      currentData = tempBuffer;
-      outBuffer = new ByteArrayOutputStream(bufferSize);
     }
 
-    outBuffer.write(currentData);
+    void dump() {
+      LOG.debug("upload block {} size: {} for blob {}",
+          entry.getId(),
+          entry.getSize(),
+          key);
+    }
   }
 
   /**
-   * Runnable instance that uploads the block of data to azure storage.
-   *
-   *
+   * Upload blob block list commands.
    */
-  private class WriteRequest implements Runnable {
-    private final byte[] dataPayload;
-    private final String blockId;
+  private class UploadBlockListCommand extends UploadCommand {
+
+    private BlockEntry lastBlock = null;
+
+    UploadBlockListCommand() {
+      super(blobLength);
 
-    public WriteRequest(byte[] dataPayload, String blockId) {
-      this.dataPayload = dataPayload;
-      this.blockId = blockId;
+      if (!uncommittedBlockEntries.isEmpty()) {
+        lastBlock = uncommittedBlockEntries.getLast();
+      }
     }
 
-    @Override
-    public void run() {
+    void awaitAsDependent() throws InterruptedException {
+      // empty. later commit block does not need to wait previous commit block
+      // lists.
+    }
 
-      int uploadRetryAttempts = 0;
-      IOException lastLocalException = null;
-      while (uploadRetryAttempts < MAX_BLOCK_UPLOAD_RETRIES) {
-        try {
+    void dump() {
+      LOG.debug("commit block list with {} blocks for blob {}",
+          uncommittedBlockEntries.size(), key);
+    }
+
+    /**
+     * Execute command.
+     */
+    public void execute() throws InterruptedException, IOException {
+
+      if (committedBlobLength.get() >= getCommandBlobOffset()) {
+        LOG.debug("commit already applied for {}", key);
+        return;
+      }
+
+      if (lastBlock == null) {
+        LOG.debug("nothing to commit for {}", key);
+        return;
+      }
+
+      LOG.debug("active commands: {} for {}", activeBlockCommands.size(), key);
+
+      for (UploadCommand activeCommand : activeBlockCommands) {
+        if (activeCommand.getCommandBlobOffset() < getCommandBlobOffset()) {
+          activeCommand.dump();
+          activeCommand.awaitAsDependent();
+        } else {
+          break;
+        }
+      }
+
+      // stop all uploads until the block list is committed
+      uploadingSemaphore.acquire(MAX_NUMBER_THREADS_IN_THREAD_POOL);
+
+      BlockEntry uncommittedBlock;
+      do  {
+        uncommittedBlock = uncommittedBlockEntries.poll();
+        blockEntries.add(uncommittedBlock);
+      } while (uncommittedBlock != lastBlock);
+
+      if (blockEntries.size() > activateCompactionBlockCount) {
+        LOG.debug("Block compaction: activated with {} blocks for {}",
+            blockEntries.size(), key);
+
+        // Block compaction
+        long startCompaction = System.nanoTime();
+        blockCompaction();
+        LOG.debug("Block compaction finished for {} ms with {} blocks for {}",
+                TimeUnit.NANOSECONDS.toMillis(
+                    System.nanoTime() - startCompaction),
+                blockEntries.size(), key);
+      }
+
+      writeBlockListRequestInternal();
+
+      uploadingSemaphore.release(MAX_NUMBER_THREADS_IN_THREAD_POOL);
 
-          blob.uploadBlock(blockId, new ByteArrayInputStream(dataPayload),
-              dataPayload.length, new BlobRequestOptions(), opContext);
+      // remove blocks previous commands
+      for (Iterator<UploadCommand> it = activeBlockCommands.iterator();
+           it.hasNext();) {
+        UploadCommand activeCommand = it.next();
+        if (activeCommand.getCommandBlobOffset() <= getCommandBlobOffset()) {
+          it.remove();
+        } else {
           break;
-        } catch(Exception ioe) {
-          Log.getLog().debug("Encountered exception during uploading block for Blob : {} Exception : {}", key, ioe);
-          uploadRetryAttempts++;
-          lastLocalException = new IOException("Encountered Exception while uploading block", ioe);
-          try {
-            Thread.sleep(BLOCK_UPLOAD_RETRY_INTERVAL);
-          } catch(InterruptedException ie) {
-            Thread.currentThread().interrupt();
-            break;
+        }
+      }
+
+      committedBlobLength.set(getCommandBlobOffset());
+    }
+
+    /**
+     * Internal output stream with read access to the internal buffer.
+     */
+    private class ByteArrayOutputStreamInternal extends ByteArrayOutputStream {
+
+      ByteArrayOutputStreamInternal(int size) {
+        super(size);
+      }
+
+      byte[] getByteArray() {
+        return buf;
+      }
+    }
+
+    /**
+     * Block compaction process.
+     *
+     * Block compaction is only enabled when the number of blocks exceeds
+     * activateCompactionBlockCount. The algorithm searches for the longest
+     * segment [b..e) where (e-b) > 2 && |b| + |b+1| ... |e-1| < maxBlockSize
+     * such that size(b1) + size(b2) + ... + size(bn) < maximum-block-size.
+     * It then downloads the blocks in the sequence, concatenates the data to
+     * form a single block, uploads this new block, and updates the block
+     * list to replace the sequence of blocks with the new block.
+     */
+    private void blockCompaction() throws IOException {
+      //current segment [segmentBegin, segmentEnd) and file offset/size of the
+      // current segment
+      int segmentBegin = 0, segmentEnd = 0;
+      long segmentOffsetBegin = 0, segmentOffsetEnd = 0;
+
+      //longest segment [maxSegmentBegin, maxSegmentEnd) and file offset/size of
+      // the longest segment
+      int maxSegmentBegin = 0, maxSegmentEnd = 0;
+      long maxSegmentOffsetBegin = 0, maxSegmentOffsetEnd = 0;
+
+      for (BlockEntry block : blockEntries) {
+        segmentEnd++;
+        segmentOffsetEnd += block.getSize();
+        if (segmentOffsetEnd - segmentOffsetBegin > maxBlockSize.get()) {
+          if (segmentEnd - segmentBegin > 2) {
+            if (maxSegmentEnd - maxSegmentBegin < segmentEnd - segmentBegin) {
+              maxSegmentBegin = segmentBegin;
+              maxSegmentEnd = segmentEnd;
+              maxSegmentOffsetBegin = segmentOffsetBegin;
+              maxSegmentOffsetEnd = segmentOffsetEnd - block.getSize();
+            }
           }
+          segmentBegin = segmentEnd - 1;
+          segmentOffsetBegin = segmentOffsetEnd - block.getSize();
         }
       }
 
-      if (uploadRetryAttempts == MAX_BLOCK_UPLOAD_RETRIES) {
-        lastError = lastLocalException;
+      if (maxSegmentEnd - maxSegmentBegin > 1) {
+
+        LOG.debug("Block compaction: {} blocks for {}",
+            maxSegmentEnd - maxSegmentBegin, key);
+
+        // download synchronously all the blocks from the azure storage
+        ByteArrayOutputStreamInternal blockOutputStream
+            = new ByteArrayOutputStreamInternal(maxBlockSize.get());
+
+        try {
+          long length = maxSegmentOffsetEnd - maxSegmentOffsetBegin;
+          blob.downloadRange(maxSegmentOffsetBegin, length, blockOutputStream,
+              new BlobRequestOptions(), opContext);
+        } catch(StorageException ex) {
+          LOG.error(
+              "Storage exception encountered during block compaction phase"
+                  + " : {} Storage Exception : {} Error Code: {}",
+              key, ex, ex.getErrorCode());
+          throw new AzureException(
+              "Encountered Exception while committing append blocks " + ex, ex);
+        }
+
+        // upload synchronously new block to the azure storage
+        String blockId = generateBlockId();
+
+        ByteBuffer byteBuffer = ByteBuffer.wrap(
+            blockOutputStream.getByteArray());
+        byteBuffer.position(blockOutputStream.size());
+
+        writeBlockRequestInternal(blockId, byteBuffer, false);
+
+        // replace blocks from the longest segment with new block id
+        blockEntries.subList(maxSegmentBegin + 1, maxSegmentEnd - 1).clear();
+        BlockEntry newBlock = blockEntries.get(maxSegmentBegin);
+        newBlock.setId(blockId);
+        newBlock.setSearchMode(BlockSearchMode.LATEST);
+        newBlock.setSize(maxSegmentOffsetEnd - maxSegmentOffsetBegin);
       }
     }
   }
 
   /**
-   * A ThreadFactory that creates uploader thread with
-   * meaningful names helpful for debugging purposes.
+   * Prepare block upload command and queue the command in thread pool executor.
    */
-  class UploaderThreadFactory implements ThreadFactory {
+  private synchronized void addBlockUploadCommand() throws IOException {
+
+    maybeThrowFirstError();
+
+    if (blobExist && lease.isFreed()) {
+      throw new AzureException(String.format(
+          "Attempting to upload a block on blob : %s "
+              + " that does not have lease on the Blob. Failing upload", key));
+    }
+
+    int blockSize = outBuffer.position();
+    if (blockSize > 0) {
+      UploadCommand command = new UploadBlockCommand(generateBlockId(),
+          outBuffer);
+      activeBlockCommands.add(command);
+
+      blobLength += blockSize;
+      outBuffer = poolReadyByteBuffers.getBuffer(false, maxBlockSize.get());
+
+      ioThreadPool.execute(new WriteRequest(command));
 
-    @Override
-    public Thread newThread(Runnable r) {
-      Thread t = new Thread(r);
-      t.setName(String.format("%s-%s-%d", THREAD_ID_PREFIX, key,
-          threadSequenceNumber.getAndIncrement()));
-      return t;
     }
   }
 
   /**
-   * A deamon thread that renews the Append lease on the blob.
-   * The thread sleeps for LEASE_RENEWAL_PERIOD time before renewing
-   * the lease. If an error is encountered while renewing the lease
-   * then an lease is released by this thread, which fails all other
-   * operations.
+   * Prepare block list commit command and queue the command in thread pool
+   * executor.
    */
-  private class AppendRenewer implements Runnable {
+  private synchronized UploadCommand addFlushCommand() throws IOException {
 
-    @Override
-    public void run() {
+    maybeThrowFirstError();
 
-      while (!leaseFreed) {
+    if (blobExist && lease.isFreed()) {
+      throw new AzureException(
+          String.format("Attempting to upload block list on blob : %s"
+              + " that does not have lease on the Blob. Failing upload", key));
+    }
 
-        try {
-          Thread.sleep(LEASE_RENEWAL_PERIOD);
-        } catch (InterruptedException ie) {
-          LOG.debug("Appender Renewer thread interrupted");
-          Thread.currentThread().interrupt();
-        }
+    UploadCommand command = new UploadBlockListCommand();
+    activeBlockCommands.add(command);
 
-        Log.getLog().debug("Attempting to renew append lease on {}", key);
+    ioThreadPool.execute(new WriteRequest(command));
 
-        try {
-          if (!leaseFreed) {
-            // Update the blob metadata to renew the append lease
-            if (!updateBlobAppendMetadata(true, true)) {
-              LOG.error("Unable to re-acquire append lease on the Blob {} ", key);
-              leaseFreed = true;
-            }
-          }
-        } catch (StorageException ex) {
+    return command;
+  }
 
-          LOG.debug("Lease renewal for Blob : {} encountered "
-              + "Storage Exception : {} Error Code : {}", key, ex, ex.getErrorCode());
+  /**
+   * Runnable instance that uploads the block of data to azure storage.
+   */
+  private class WriteRequest implements Runnable {
+    private final UploadCommand command;
 
-          // We swallow the exception here because if the blob metadata is not updated for
-          // APPEND_LEASE_TIMEOUT period, another thread would be able to detect this and
-          // continue forward if it needs to append.
-          leaseFreed = true;
-        }
+    WriteRequest(UploadCommand command) {
+      this.command = command;
+    }
+
+    @Override
+    public void run() {
+
+      try {
+        command.dump();
+        long startTime = System.nanoTime();
+        command.execute();
+        command.setCompleted();
+        LOG.debug("command finished for {} ms",
+            TimeUnit.NANOSECONDS.toMillis(
+                    System.nanoTime() - startTime));
+      } catch (InterruptedException ie) {
+        Thread.currentThread().interrupt();
+      } catch (Exception ex) {
+        LOG.debug(
+                "Encountered exception during execution of command for Blob :"
+                        + " {} Exception : {}", key, ex);
+        firstError.compareAndSet(null, new AzureException(ex));
       }
     }
   }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/13eda500/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/NativeAzureFileSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/NativeAzureFileSystem.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/NativeAzureFileSystem.java
index 0bde124..280c0e0 100644
--- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/NativeAzureFileSystem.java
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/NativeAzureFileSystem.java
@@ -62,6 +62,8 @@ import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Seekable;
+import org.apache.hadoop.fs.StreamCapabilities;
+import org.apache.hadoop.fs.Syncable;
 import org.apache.hadoop.fs.azure.metrics.AzureFileSystemInstrumentation;
 import org.apache.hadoop.fs.azure.metrics.AzureFileSystemMetricsSystem;
 import org.apache.hadoop.fs.azure.security.Constants;
@@ -352,9 +354,9 @@ public class NativeAzureFileSystem extends FileSystem {
     }
 
     /**
-     * This is an exact copy of org.codehaus.jettison.json.JSONObject.quote 
+     * This is an exact copy of org.codehaus.jettison.json.JSONObject.quote
      * method.
-     * 
+     *
      * Produce a string in double quotes with backslash sequences in all the
      * right places. A backslash will be inserted within </, allowing JSON
      * text to be delivered in HTML. In JSON text, a string cannot contain a
@@ -947,11 +949,11 @@ public class NativeAzureFileSystem extends FileSystem {
     }
   }
 
-  private class NativeAzureFsOutputStream extends OutputStream {
-    // We should not override flush() to actually close current block and flush
-    // to DFS, this will break applications that assume flush() is a no-op.
-    // Applications are advised to use Syncable.hflush() for that purpose.
-    // NativeAzureFsOutputStream needs to implement Syncable if needed.
+  /**
+   * Azure output stream; wraps an inner stream of different types.
+   */
+  public class NativeAzureFsOutputStream extends OutputStream
+      implements Syncable, StreamCapabilities {
     private String key;
     private String keyEncoded;
     private OutputStream out;
@@ -983,6 +985,48 @@ public class NativeAzureFileSystem extends FileSystem {
       setEncodedKey(anEncodedKey);
     }
 
+    /**
+     * Get a reference to the wrapped output stream.
+     *
+     * @return the underlying output stream
+     */
+    @InterfaceAudience.LimitedPrivate({"HDFS"})
+    public OutputStream getOutStream() {
+      return out;
+    }
+
+    @Override  // Syncable
+    public void hflush() throws IOException {
+      if (out instanceof Syncable) {
+        ((Syncable) out).hflush();
+      } else {
+        flush();
+      }
+    }
+
+    @Override  // Syncable
+    public void hsync() throws IOException {
+      if (out instanceof Syncable) {
+        ((Syncable) out).hsync();
+      } else {
+        flush();
+      }
+    }
+
+    /**
+     * Propagate probe of stream capabilities to nested stream
+     * (if supported), else return false.
+     * @param capability string to query the stream support for.
+     * @return true if the nested stream supports the specific capability.
+     */
+    @Override // StreamCapability
+    public boolean hasCapability(String capability) {
+      if (out instanceof StreamCapabilities) {
+        return ((StreamCapabilities) out).hasCapability(capability);
+      }
+      return false;
+    }
+
     @Override
     public synchronized void close() throws IOException {
       if (out != null) {
@@ -990,8 +1034,11 @@ public class NativeAzureFileSystem extends FileSystem {
         // before returning to the caller.
         //
         out.close();
-        restoreKey();
-        out = null;
+        try {
+          restoreKey();
+        } finally {
+          out = null;
+        }
       }
     }
 
@@ -1045,10 +1092,10 @@ public class NativeAzureFileSystem extends FileSystem {
     /**
      * Writes <code>len</code> from the specified byte array starting at offset
      * <code>off</code> to the output stream. The general contract for write(b,
-     * off, len) is that some of the bytes in the array <code>
-     * b</code b> are written to the output stream in order; element
-     * <code>b[off]</code> is the first byte written and
-     * <code>b[off+len-1]</code> is the last byte written by this operation.
+     * off, len) is that some of the bytes in the array <code>b</code>
+     * are written to the output stream in order; element <code>b[off]</code>
+     * is the first byte written and <code>b[off+len-1]</code> is the last
+     * byte written by this operation.
      * 
      * @param b
      *          Byte array to be written.
@@ -1749,7 +1796,7 @@ public class NativeAzureFileSystem extends FileSystem {
     OutputStream bufOutStream;
     if (store.isPageBlobKey(key)) {
       // Store page blobs directly in-place without renames.
-      bufOutStream = store.storefile(key, permissionStatus);
+      bufOutStream = store.storefile(key, permissionStatus, key);
     } else {
       // This is a block blob, so open the output blob stream based on the
       // encoded key.
@@ -1777,7 +1824,7 @@ public class NativeAzureFileSystem extends FileSystem {
       // these
       // blocks.
       bufOutStream = new NativeAzureFsOutputStream(store.storefile(
-          keyEncoded, permissionStatus), key, keyEncoded);
+          keyEncoded, permissionStatus, key), key, keyEncoded);
     }
     // Construct the data output stream from the buffered output stream.
     FSDataOutputStream fsOut = new FSDataOutputStream(bufOutStream, statistics);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/13eda500/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/NativeFileSystemStore.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/NativeFileSystemStore.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/NativeFileSystemStore.java
index 1c7309f..57a729d 100644
--- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/NativeFileSystemStore.java
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/NativeFileSystemStore.java
@@ -50,8 +50,9 @@ interface NativeFileSystemStore {
 
   InputStream retrieve(String key, long byteRangeStart) throws IOException;
 
-  DataOutputStream storefile(String key, PermissionStatus permissionStatus)
-      throws AzureException;
+  DataOutputStream storefile(String keyEncoded,
+      PermissionStatus permissionStatus,
+      String key) throws AzureException;
 
   boolean isPageBlobKey(String key);
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/13eda500/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/SecureStorageInterfaceImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/SecureStorageInterfaceImpl.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/SecureStorageInterfaceImpl.java
index 5dbb6bc..7c2722e 100644
--- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/SecureStorageInterfaceImpl.java
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/SecureStorageInterfaceImpl.java
@@ -519,7 +519,7 @@ public class SecureStorageInterfaceImpl extends StorageInterface {
 
     @Override
     public SelfRenewingLease acquireLease() throws StorageException {
-      return new SelfRenewingLease(this);
+      return new SelfRenewingLease(this, false);
     }
   }
 
@@ -557,10 +557,12 @@ public class SecureStorageInterfaceImpl extends StorageInterface {
     }
 
     @Override
-    public void uploadBlock(String blockId, InputStream sourceStream,
+    public void uploadBlock(String blockId, AccessCondition accessCondition,
+        InputStream sourceStream,
         long length, BlobRequestOptions options,
         OperationContext opContext) throws IOException, StorageException {
-      ((CloudBlockBlob) getBlob()).uploadBlock(blockId, sourceStream, length, null, options, opContext);
+      ((CloudBlockBlob) getBlob()).uploadBlock(blockId, sourceStream, length,
+          accessCondition, options, opContext);
     }
 
     @Override
@@ -593,4 +595,4 @@ public class SecureStorageInterfaceImpl extends StorageInterface {
           null, options, opContext);
     }
   }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/13eda500/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/SelfRenewingLease.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/SelfRenewingLease.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/SelfRenewingLease.java
index 00d5e99..10956f7 100644
--- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/SelfRenewingLease.java
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/SelfRenewingLease.java
@@ -30,6 +30,8 @@ import com.microsoft.azure.storage.blob.CloudBlob;
 
 import java.util.concurrent.atomic.AtomicInteger;
 
+import static com.microsoft.azure.storage.StorageErrorCodeStrings.LEASE_ALREADY_PRESENT;
+
 /**
  * An Azure blob lease that automatically renews itself indefinitely
  * using a background thread. Use it to synchronize distributed processes,
@@ -66,7 +68,7 @@ public class SelfRenewingLease {
   @VisibleForTesting
   static final int LEASE_ACQUIRE_RETRY_INTERVAL = 2000;
 
-  public SelfRenewingLease(CloudBlobWrapper blobWrapper)
+  public SelfRenewingLease(CloudBlobWrapper blobWrapper, boolean throwIfPresent)
       throws StorageException {
 
     this.leaseFreed = false;
@@ -79,10 +81,14 @@ public class SelfRenewingLease {
         leaseID = blob.acquireLease(LEASE_TIMEOUT, null);
       } catch (StorageException e) {
 
+        if (throwIfPresent && e.getErrorCode().equals(LEASE_ALREADY_PRESENT)) {
+          throw e;
+        }
+
         // Throw again if we don't want to keep waiting.
         // We expect it to be that the lease is already present,
         // or in some cases that the blob does not exist.
-        if (!"LeaseAlreadyPresent".equals(e.getErrorCode())) {
+        if (!LEASE_ALREADY_PRESENT.equals(e.getErrorCode())) {
           LOG.info(
             "Caught exception when trying to get lease on blob "
             + blobWrapper.getUri().toString() + ". " + e.getMessage());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/13eda500/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/StorageInterface.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/StorageInterface.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/StorageInterface.java
index 8b6b082..e03d731 100644
--- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/StorageInterface.java
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/StorageInterface.java
@@ -665,6 +665,7 @@ abstract class StorageInterface {
      *
      * @param blockId      A String that represents the Base-64 encoded block ID. Note for a given blob
      *                     the length of all Block IDs must be identical.
+     * @param accessCondition An {@link AccessCondition} object that represents the access conditions for the blob.
      * @param sourceStream An {@link InputStream} object that represents the input stream to write to the
      *                     block blob.
      * @param length       A long which represents the length, in bytes, of the stream data,
@@ -678,7 +679,7 @@ abstract class StorageInterface {
      * @throws IOException  If an I/O error occurred.
      * @throws StorageException If a storage service error occurred.
      */
-    void uploadBlock(String blockId, InputStream sourceStream,
+    void uploadBlock(String blockId, AccessCondition accessCondition, InputStream sourceStream,
         long length, BlobRequestOptions options,
         OperationContext opContext) throws IOException, StorageException;
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/13eda500/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/StorageInterfaceImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/StorageInterfaceImpl.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/StorageInterfaceImpl.java
index d3d0370..41a4dbb 100644
--- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/StorageInterfaceImpl.java
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/StorageInterfaceImpl.java
@@ -277,7 +277,7 @@ class StorageInterfaceImpl extends StorageInterface {
 
       return new CloudBlockBlobWrapperImpl(container.getBlockBlobReference(relativePath));
     }
-    
+
     @Override
     public CloudBlobWrapper getPageBlobReference(String relativePath)
         throws URISyntaxException, StorageException {
@@ -286,7 +286,7 @@ class StorageInterfaceImpl extends StorageInterface {
     }
 
   }
-  
+
   abstract static class CloudBlobWrapperImpl implements CloudBlobWrapper {
     private final CloudBlob blob;
 
@@ -441,10 +441,10 @@ class StorageInterfaceImpl extends StorageInterface {
 
     @Override
     public SelfRenewingLease acquireLease() throws StorageException {
-      return new SelfRenewingLease(this);
+      return new SelfRenewingLease(this, false);
     }
   }
-  
+
 
   //
   // CloudBlockBlobWrapperImpl
@@ -479,10 +479,10 @@ class StorageInterfaceImpl extends StorageInterface {
     }
 
     @Override
-    public void uploadBlock(String blockId, InputStream sourceStream,
+    public void uploadBlock(String blockId, AccessCondition accessCondition, InputStream sourceStream,
         long length, BlobRequestOptions options,
         OperationContext opContext) throws IOException, StorageException {
-      ((CloudBlockBlob) getBlob()).uploadBlock(blockId, sourceStream, length, null, options, opContext);
+      ((CloudBlockBlob) getBlob()).uploadBlock(blockId, sourceStream, length, accessCondition, options, opContext);
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/hadoop/blob/13eda500/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/SyncableDataOutputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/SyncableDataOutputStream.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/SyncableDataOutputStream.java
index a52fdb7..fc8796b 100644
--- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/SyncableDataOutputStream.java
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/SyncableDataOutputStream.java
@@ -24,6 +24,7 @@ import java.io.OutputStream;
 
 import org.apache.hadoop.fs.StreamCapabilities;
 import org.apache.hadoop.fs.Syncable;
+import org.apache.hadoop.classification.InterfaceAudience;
 
 /**
  * Support the Syncable interface on top of a DataOutputStream.
@@ -38,6 +39,16 @@ public class SyncableDataOutputStream extends DataOutputStream
     super(out);
   }
 
+  /**
+   * Get a reference to the wrapped output stream.
+   *
+   * @return the underlying output stream
+   */
+  @InterfaceAudience.LimitedPrivate({"HDFS"})
+  public OutputStream getOutStream() {
+    return out;
+  }
+
   @Override
   public boolean hasCapability(String capability) {
     if (out instanceof StreamCapabilities) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/13eda500/hadoop-tools/hadoop-azure/src/site/markdown/index.md
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/site/markdown/index.md b/hadoop-tools/hadoop-azure/src/site/markdown/index.md
index 758650d..466bf0b 100644
--- a/hadoop-tools/hadoop-azure/src/site/markdown/index.md
+++ b/hadoop-tools/hadoop-azure/src/site/markdown/index.md
@@ -153,6 +153,40 @@ line argument:
 
 ```
 
+### Block Blob with Compaction Support and Configuration
+
+Block blobs are the default kind of blob and are good for most big-data use
+cases. However, block blobs have strict limit of 50,000 blocks per blob.
+To prevent reaching the limit WASB, by default, does not upload new block to
+the service after every `hflush()` or `hsync()`.
+
+For most of the cases, combining data from multiple `write()` calls in
+blocks of 4Mb is a good optimization. But, in others cases, like HBase log files,
+every call to `hflush()` or `hsync()` must upload the data to the service.
+
+Block blobs with compaction upload the data to the cloud service after every
+`hflush()`/`hsync()`. To mitigate the limit of 50000 blocks, `hflush()
+`/`hsync()` runs once compaction process, if number of blocks in the blob
+is above 32,000.
+
+Block compaction search and replaces a sequence of small blocks with one big
+block. That means there is associated cost with block compaction: reading
+small blocks back to the client and writing it again as one big block.
+
+In order to have the files you create be block blobs with block compaction
+enabled, the client must set the configuration variable
+`fs.azure.block.blob.with.compaction.dir` to a comma-separated list of
+folder names.
+
+For example:
+
+```xml
+<property>
+  <name>fs.azure.block.blob.with.compaction.dir</name>
+  <value>/hbase/WALs,/data/myblobfiles</value>
+</property>
+```
+
 ### Page Blob Support and Configuration
 
 The Azure Blob Storage interface for Hadoop supports two kinds of blobs,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/13eda500/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/MockStorageInterface.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/MockStorageInterface.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/MockStorageInterface.java
index 4f26d9f..e0ae7b4 100644
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/MockStorageInterface.java
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/MockStorageInterface.java
@@ -551,7 +551,8 @@ public class MockStorageInterface extends StorageInterface {
       throw new UnsupportedOperationException("downloadBlockList not used in Mock Tests");
     }
     @Override
-    public void uploadBlock(String blockId, InputStream sourceStream,
+    public void uploadBlock(String blockId, AccessCondition accessCondition,
+        InputStream sourceStream,
         long length, BlobRequestOptions options,
         OperationContext opContext) throws IOException, StorageException {
       throw new UnsupportedOperationException("uploadBlock not used in Mock Tests");

http://git-wip-us.apache.org/repos/asf/hadoop/blob/13eda500/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestAzureConcurrentOutOfBandIo.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestAzureConcurrentOutOfBandIo.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestAzureConcurrentOutOfBandIo.java
index 7ea7534..a10a366 100644
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestAzureConcurrentOutOfBandIo.java
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestAzureConcurrentOutOfBandIo.java
@@ -107,7 +107,8 @@ public class TestAzureConcurrentOutOfBandIo {
           //
           outputStream = writerStorageAccount.getStore().storefile(
               key,
-              new PermissionStatus("", "", FsPermission.getDefault()));
+              new PermissionStatus("", "", FsPermission.getDefault()),
+              key);
 
           Arrays.fill(dataBlockWrite, (byte) (i % 256));
           for (int j = 0; j < NUMBER_OF_BLOCKS; j++) {
@@ -141,7 +142,8 @@ public class TestAzureConcurrentOutOfBandIo {
    // reading.  This eliminates the race between the reader and writer threads.
    OutputStream outputStream = testAccount.getStore().storefile(
        "WASB_String.txt",
-       new PermissionStatus("", "", FsPermission.getDefault()));
+       new PermissionStatus("", "", FsPermission.getDefault()),
+           "WASB_String.txt");
    Arrays.fill(dataBlockWrite, (byte) 255);
    for (int i = 0; i < NUMBER_OF_BLOCKS; i++) {
      outputStream.write(dataBlockWrite);


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[21/37] hadoop git commit: YARN-7144. Log Aggregation controller should not swallow the exceptions when it calls closeWriter and closeReader. Contributed by Xuan Gong.

Posted by ae...@apache.org.
YARN-7144. Log Aggregation controller should not swallow the exceptions when it calls closeWriter and closeReader. Contributed by Xuan Gong.


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

Branch: refs/heads/HDFS-7240
Commit: 22de9449f8aa72c5b0bb586b8253390773502189
Parents: dd81494
Author: Junping Du <ju...@apache.org>
Authored: Wed Sep 6 14:53:31 2017 -0700
Committer: Junping Du <ju...@apache.org>
Committed: Wed Sep 6 14:53:31 2017 -0700

----------------------------------------------------------------------
 .../logaggregation/AggregatedLogFormat.java     | 27 +++++++++-----------
 .../LogAggregationFileController.java           | 10 ++++----
 2 files changed, 17 insertions(+), 20 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/22de9449/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/AggregatedLogFormat.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/AggregatedLogFormat.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/AggregatedLogFormat.java
index 9bec147..af3066e 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/AggregatedLogFormat.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/AggregatedLogFormat.java
@@ -44,11 +44,8 @@ import java.util.Map;
 import java.util.Map.Entry;
 import java.util.Set;
 import java.util.regex.Pattern;
-import org.apache.commons.io.IOUtils;
 import org.apache.commons.io.input.BoundedInputStream;
 import org.apache.commons.io.output.WriterOutputStream;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.commons.math3.util.Pair;
 import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.classification.InterfaceAudience.Public;
@@ -61,6 +58,7 @@ import org.apache.hadoop.fs.FileContext;
 import org.apache.hadoop.fs.Options;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.io.SecureIOUtils;
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.io.file.tfile.TFile;
@@ -71,7 +69,8 @@ import org.apache.hadoop.yarn.api.records.LogAggregationContext;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
 import org.apache.hadoop.yarn.util.Times;
-
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Predicate;
 import com.google.common.collect.Iterables;
@@ -81,7 +80,8 @@ import com.google.common.collect.Sets;
 @Evolving
 public class AggregatedLogFormat {
 
-  private static final Log LOG = LogFactory.getLog(AggregatedLogFormat.class);
+  private final static Logger LOG = LoggerFactory.getLogger(
+      AggregatedLogFormat.class);
   private static final LogKey APPLICATION_ACL_KEY = new LogKey("APPLICATION_ACL");
   private static final LogKey APPLICATION_OWNER_KEY = new LogKey("APPLICATION_OWNER");
   private static final LogKey VERSION_KEY = new LogKey("VERSION");
@@ -247,7 +247,7 @@ public class AggregatedLogFormat {
           in = secureOpenFile(logFile);
         } catch (IOException e) {
           logErrorMessage(logFile, e);
-          IOUtils.closeQuietly(in);
+          IOUtils.cleanupWithLogger(LOG, in);
           continue;
         }
 
@@ -285,7 +285,7 @@ public class AggregatedLogFormat {
           String message = logErrorMessage(logFile, e);
           out.write(message.getBytes(Charset.forName("UTF-8")));
         } finally {
-          IOUtils.closeQuietly(in);
+          IOUtils.cleanupWithLogger(LOG, in);
         }
       }
     }
@@ -555,7 +555,7 @@ public class AggregatedLogFormat {
       } catch (Exception e) {
         LOG.warn("Exception closing writer", e);
       } finally {
-        IOUtils.closeQuietly(this.fsDataOStream);
+        IOUtils.cleanupWithLogger(LOG, this.fsDataOStream);
       }
     }
   }
@@ -603,7 +603,7 @@ public class AggregatedLogFormat {
         }
         return null;
       } finally {
-        IOUtils.closeQuietly(ownerScanner);
+        IOUtils.cleanupWithLogger(LOG, ownerScanner);
       }
     }
 
@@ -649,7 +649,7 @@ public class AggregatedLogFormat {
         }
         return acls;
       } finally {
-        IOUtils.closeQuietly(aclScanner);
+        IOUtils.cleanupWithLogger(LOG, aclScanner);
       }
     }
 
@@ -774,8 +774,7 @@ public class AggregatedLogFormat {
           }
         }
       } finally {
-        IOUtils.closeQuietly(ps);
-        IOUtils.closeQuietly(os);
+        IOUtils.cleanupWithLogger(LOG, ps, os);
       }
     }
 
@@ -1001,9 +1000,7 @@ public class AggregatedLogFormat {
     }
 
     public void close() {
-      IOUtils.closeQuietly(scanner);
-      IOUtils.closeQuietly(reader);
-      IOUtils.closeQuietly(fsDataIStream);
+      IOUtils.cleanupWithLogger(LOG, scanner, reader, fsDataIStream);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/22de9449/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/filecontroller/LogAggregationFileController.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/filecontroller/LogAggregationFileController.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/filecontroller/LogAggregationFileController.java
index aafdb66..39f3dc3 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/filecontroller/LogAggregationFileController.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/filecontroller/LogAggregationFileController.java
@@ -37,9 +37,6 @@ import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
-import org.apache.commons.io.IOUtils;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.conf.Configuration;
@@ -47,6 +44,7 @@ import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.yarn.api.records.ApplicationAccessType;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
@@ -56,6 +54,8 @@ import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
 import org.apache.hadoop.yarn.logaggregation.LogAggregationUtils;
 import org.apache.hadoop.yarn.webapp.View.ViewContext;
 import org.apache.hadoop.yarn.webapp.view.HtmlBlock.Block;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 import org.apache.hadoop.yarn.logaggregation.AggregatedLogFormat.LogKey;
 import org.apache.hadoop.yarn.logaggregation.AggregatedLogFormat.LogValue;
 import org.apache.hadoop.yarn.logaggregation.ContainerLogMeta;
@@ -68,7 +68,7 @@ import org.apache.hadoop.yarn.logaggregation.ContainerLogsRequest;
 @Unstable
 public abstract class LogAggregationFileController {
 
-  private static final Log LOG = LogFactory.getLog(
+  private static final Logger LOG = LoggerFactory.getLogger(
       LogAggregationFileController.class);
 
   /*
@@ -193,7 +193,7 @@ public abstract class LogAggregationFileController {
 
   protected void closePrintStream(OutputStream out) {
     if (out != System.out) {
-      IOUtils.closeQuietly(out);
+      IOUtils.cleanupWithLogger(LOG, out);
     }
   }
 


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[29/37] hadoop git commit: HDFS-12402. Refactor ErasureCodingPolicyManager and related codes. Contributed by Sammi Chen

Posted by ae...@apache.org.
HDFS-12402. Refactor ErasureCodingPolicyManager and related codes. Contributed by Sammi Chen


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

Branch: refs/heads/HDFS-7240
Commit: 2adf8bed712e6d770a0d53eea198d8911ae1a258
Parents: 6f101e7
Author: Kai Zheng <ka...@intel.com>
Authored: Thu Sep 7 20:38:23 2017 +0800
Committer: Kai Zheng <ka...@intel.com>
Committed: Thu Sep 7 20:38:23 2017 +0800

----------------------------------------------------------------------
 .../hdfs/protocol/AddECPolicyResponse.java      |  4 +-
 .../hdfs/protocol/IllegalECPolicyException.java | 34 -----------------
 .../namenode/ErasureCodingPolicyManager.java    | 39 ++++++++++----------
 .../server/namenode/FSDirErasureCodingOp.java   |  3 +-
 .../hdfs/server/namenode/FSNamesystem.java      |  3 +-
 .../hadoop/hdfs/TestDistributedFileSystem.java  |  4 +-
 6 files changed, 27 insertions(+), 60 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/2adf8bed/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/AddECPolicyResponse.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/AddECPolicyResponse.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/AddECPolicyResponse.java
index e7a8435..4e55680 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/AddECPolicyResponse.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/AddECPolicyResponse.java
@@ -17,6 +17,8 @@
  */
 package org.apache.hadoop.hdfs.protocol;
 
+import org.apache.hadoop.HadoopIllegalArgumentException;
+
 /**
  * A response of add an ErasureCoding policy.
  */
@@ -38,7 +40,7 @@ public class AddECPolicyResponse {
   }
 
   public AddECPolicyResponse(ErasureCodingPolicy policy,
-      IllegalECPolicyException e) {
+      HadoopIllegalArgumentException e) {
     this(policy, e.getMessage());
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2adf8bed/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/IllegalECPolicyException.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/IllegalECPolicyException.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/IllegalECPolicyException.java
deleted file mode 100644
index 03ce2a5..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/IllegalECPolicyException.java
+++ /dev/null
@@ -1,34 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hdfs.protocol;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-
-/**
- * An Exception indicates the error when adding an ErasureCoding policy.
- */
-@InterfaceAudience.Private
-@InterfaceStability.Evolving
-public class IllegalECPolicyException extends Exception {
-  static final long serialVersionUID = 1L;
-
-  public IllegalECPolicyException(String msg) {
-    super(msg);
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2adf8bed/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ErasureCodingPolicyManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ErasureCodingPolicyManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ErasureCodingPolicyManager.java
index 74b5ebf..4c75709 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ErasureCodingPolicyManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ErasureCodingPolicyManager.java
@@ -19,11 +19,11 @@ package org.apache.hadoop.hdfs.server.namenode;
 
 import com.google.common.annotations.VisibleForTesting;
 import org.apache.commons.lang.ArrayUtils;
+import org.apache.hadoop.HadoopIllegalArgumentException;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicyState;
-import org.apache.hadoop.hdfs.protocol.IllegalECPolicyException;
 import org.apache.hadoop.hdfs.protocol.SystemErasureCodingPolicies;
 import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
@@ -144,7 +144,7 @@ public final class ErasureCodingPolicyManager {
             policyName,
             DFSConfigKeys.DFS_NAMENODE_EC_POLICIES_ENABLED_KEY,
             names);
-        throw new IllegalArgumentException(msg);
+        throw new HadoopIllegalArgumentException(msg);
       }
       enabledPoliciesByName.put(ecPolicy.getName(), ecPolicy);
     }
@@ -230,33 +230,34 @@ public final class ErasureCodingPolicyManager {
    * Add an erasure coding policy.
    * @return the added policy
    */
-  public synchronized ErasureCodingPolicy addPolicy(ErasureCodingPolicy policy)
-      throws IllegalECPolicyException {
+  public synchronized ErasureCodingPolicy addPolicy(
+      ErasureCodingPolicy policy) {
     // Set policy state into DISABLED when adding into Hadoop.
     policy.setState(ErasureCodingPolicyState.DISABLED);
 
     if (!CodecUtil.hasCodec(policy.getCodecName())) {
-      throw new IllegalECPolicyException("Codec name "
+      throw new HadoopIllegalArgumentException("Codec name "
           + policy.getCodecName() + " is not supported");
     }
 
     if (policy.getCellSize() > maxCellSize) {
-      throw new IllegalECPolicyException("Cell size " + policy.getCellSize()
-          + " should not exceed maximum " + maxCellSize + " byte");
+      throw new HadoopIllegalArgumentException("Cell size " +
+          policy.getCellSize() + " should not exceed maximum " +
+          maxCellSize + " bytes");
     }
 
     String assignedNewName = ErasureCodingPolicy.composePolicyName(
         policy.getSchema(), policy.getCellSize());
     for (ErasureCodingPolicy p : getPolicies()) {
       if (p.getName().equals(assignedNewName)) {
-        throw new IllegalECPolicyException("The policy name " + assignedNewName
-            + " already exists");
+        throw new HadoopIllegalArgumentException("The policy name " +
+            assignedNewName + " already exists");
       }
       if (p.getSchema().equals(policy.getSchema()) &&
           p.getCellSize() == policy.getCellSize()) {
-        throw new IllegalECPolicyException("A policy with same schema "
+        throw new HadoopIllegalArgumentException("A policy with same schema "
             + policy.getSchema().toString() + " and cell size "
-            + p.getCellSize() + " is already exists");
+            + p.getCellSize() + " already exists");
       }
     }
     policy.setName(assignedNewName);
@@ -281,12 +282,12 @@ public final class ErasureCodingPolicyManager {
   public synchronized void removePolicy(String name) {
     ErasureCodingPolicy ecPolicy = policiesByName.get(name);
     if (ecPolicy == null) {
-      throw new IllegalArgumentException("The policy name " +
-          name + " does not exists");
+      throw new HadoopIllegalArgumentException("The policy name " +
+          name + " does not exist");
     }
 
     if (ecPolicy.isSystemPolicy()) {
-      throw new IllegalArgumentException("System erasure coding policy " +
+      throw new HadoopIllegalArgumentException("System erasure coding policy " +
           name + " cannot be removed");
     }
 
@@ -317,8 +318,8 @@ public final class ErasureCodingPolicyManager {
   public synchronized void disablePolicy(String name) {
     ErasureCodingPolicy ecPolicy = policiesByName.get(name);
     if (ecPolicy == null) {
-      throw new IllegalArgumentException("The policy name " +
-          name + " does not exists");
+      throw new HadoopIllegalArgumentException("The policy name " +
+          name + " does not exist");
     }
 
     if (enabledPoliciesByName.containsKey(name)) {
@@ -336,8 +337,8 @@ public final class ErasureCodingPolicyManager {
   public synchronized void enablePolicy(String name) {
     ErasureCodingPolicy ecPolicy = policiesByName.get(name);
     if (ecPolicy == null) {
-      throw new IllegalArgumentException("The policy name " +
-          name + " does not exists");
+      throw new HadoopIllegalArgumentException("The policy name " +
+          name + " does not exist");
     }
 
     enabledPoliciesByName.put(name, ecPolicy);
@@ -346,4 +347,4 @@ public final class ErasureCodingPolicyManager {
         enabledPoliciesByName.values().toArray(new ErasureCodingPolicy[0]);
     LOG.info("Enable the erasure coding policy " + name);
   }
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2adf8bed/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirErasureCodingOp.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirErasureCodingOp.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirErasureCodingOp.java
index 4f4befe..deb03af 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirErasureCodingOp.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirErasureCodingOp.java
@@ -27,7 +27,6 @@ import org.apache.hadoop.fs.permission.FsAction;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.XAttrHelper;
 import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
-import org.apache.hadoop.hdfs.protocol.IllegalECPolicyException;
 import org.apache.hadoop.hdfs.server.namenode.FSDirectory.DirOp;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.io.WritableUtils;
@@ -212,7 +211,7 @@ final class FSDirErasureCodingOp {
   }
 
   static ErasureCodingPolicy addErasureCodePolicy(final FSNamesystem fsn,
-      ErasureCodingPolicy policy) throws IllegalECPolicyException {
+      ErasureCodingPolicy policy) {
     Preconditions.checkNotNull(policy);
     return fsn.getErasureCodingPolicyManager().addPolicy(policy);
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2adf8bed/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
index e5604c4..c30999b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
@@ -201,7 +201,6 @@ import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.ReencryptAction;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
-import org.apache.hadoop.hdfs.protocol.IllegalECPolicyException;
 import org.apache.hadoop.hdfs.protocol.LastBlockWithStatus;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
@@ -7207,7 +7206,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
               FSDirErasureCodingOp.addErasureCodePolicy(this, policy);
           addECPolicyName = newPolicy.getName();
           responses.add(new AddECPolicyResponse(newPolicy));
-        } catch (IllegalECPolicyException e) {
+        } catch (HadoopIllegalArgumentException e) {
           responses.add(new AddECPolicyResponse(policy, e));
         }
       }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2adf8bed/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDistributedFileSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDistributedFileSystem.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDistributedFileSystem.java
index bf2b002..8e54e5f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDistributedFileSystem.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDistributedFileSystem.java
@@ -1617,7 +1617,7 @@ public class TestDistributedFileSystem {
         fs.enableErasureCodingPolicy("notExistECName");
         Assert.fail("enable the policy that doesn't exist should fail");
       } catch (Exception e) {
-        GenericTestUtils.assertExceptionContains("does not exists", e);
+        GenericTestUtils.assertExceptionContains("does not exist", e);
         // pass
       }
 
@@ -1626,7 +1626,7 @@ public class TestDistributedFileSystem {
         fs.disableErasureCodingPolicy("notExistECName");
         Assert.fail("disable the policy that doesn't exist should fail");
       } catch (Exception e) {
-        GenericTestUtils.assertExceptionContains("does not exists", e);
+        GenericTestUtils.assertExceptionContains("does not exist", e);
         // pass
       }
 


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[23/37] hadoop git commit: Revert "Plan/ResourceAllocation data structure enhancements required to support recurring reservations in ReservationSystem."

Posted by ae...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/e3345e98/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/RLESparseResourceAllocation.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/reservation/RLESparseResourceAllocation.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/RLESparseResourceAllocation.java
index 100d38c..658387b 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/RLESparseResourceAllocation.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/RLESparseResourceAllocation.java
@@ -18,7 +18,8 @@
 
 package org.apache.hadoop.yarn.server.resourcemanager.reservation;
 
-import java.util.Collections;
+import java.io.IOException;
+import java.io.StringWriter;
 import java.util.Iterator;
 import java.util.Map;
 import java.util.Map.Entry;
@@ -32,6 +33,8 @@ import org.apache.hadoop.yarn.server.resourcemanager.reservation.exceptions.Plan
 import org.apache.hadoop.yarn.util.resource.ResourceCalculator;
 import org.apache.hadoop.yarn.util.resource.Resources;
 
+import com.google.gson.stream.JsonWriter;
+
 /**
  * This is a run length encoded sparse data structure that maintains resource
  * allocations over time.
@@ -41,14 +44,12 @@ public class RLESparseResourceAllocation {
   private static final int THRESHOLD = 100;
   private static final Resource ZERO_RESOURCE = Resources.none();
 
-  @SuppressWarnings("checkstyle:visibilitymodifier")
-  protected NavigableMap<Long, Resource> cumulativeCapacity =
+  private NavigableMap<Long, Resource> cumulativeCapacity =
       new TreeMap<Long, Resource>();
 
   private final ReentrantReadWriteLock readWriteLock =
       new ReentrantReadWriteLock();
-  @SuppressWarnings("checkstyle:visibilitymodifier")
-  protected final Lock readLock = readWriteLock.readLock();
+  private final Lock readLock = readWriteLock.readLock();
   private final Lock writeLock = readWriteLock.writeLock();
 
   private final ResourceCalculator resourceCalculator;
@@ -235,6 +236,34 @@ public class RLESparseResourceAllocation {
   }
 
   /**
+   * Returns the JSON string representation of the current resources allocated
+   * over time.
+   *
+   * @return the JSON string representation of the current resources allocated
+   *         over time
+   */
+  public String toMemJSONString() {
+    StringWriter json = new StringWriter();
+    JsonWriter jsonWriter = new JsonWriter(json);
+    readLock.lock();
+    try {
+      jsonWriter.beginObject();
+      // jsonWriter.name("timestamp").value("resource");
+      for (Map.Entry<Long, Resource> r : cumulativeCapacity.entrySet()) {
+        jsonWriter.name(r.getKey().toString()).value(r.getValue().toString());
+      }
+      jsonWriter.endObject();
+      jsonWriter.close();
+      return json.toString();
+    } catch (IOException e) {
+      // This should not happen
+      return "";
+    } finally {
+      readLock.unlock();
+    }
+  }
+
+  /**
    * Returns the representation of the current resources allocated over time as
    * an interval map (in the defined non-null range).
    *
@@ -275,7 +304,7 @@ public class RLESparseResourceAllocation {
   public NavigableMap<Long, Resource> getCumulative() {
     readLock.lock();
     try {
-      return Collections.unmodifiableNavigableMap(cumulativeCapacity);
+      return cumulativeCapacity;
     } finally {
       readLock.unlock();
     }
@@ -408,8 +437,8 @@ public class RLESparseResourceAllocation {
       Resource val = Resources.negate(e.getValue());
       // test for negative value and throws
       if (operator == RLEOperator.subtractTestNonNegative
-          && (Resources.fitsIn(val, ZERO_RESOURCE)
-              && !Resources.equals(val, ZERO_RESOURCE))) {
+          && (Resources.fitsIn(val, ZERO_RESOURCE) &&
+              !Resources.equals(val, ZERO_RESOURCE))) {
         throw new PlanningException(
             "RLESparseResourceAllocation: merge failed as the "
                 + "resulting RLESparseResourceAllocation would be negative");
@@ -475,29 +504,22 @@ public class RLESparseResourceAllocation {
 
   }
 
-  /**
-   * Get a {@link RLESparseResourceAllocation} view of the {@link Resource}
-   * allocations between the specified start and end times.
-   *
-   * @param start the time from which the {@link Resource} allocations are
-   *          required
-   * @param end the time upto which the {@link Resource} allocations are
-   *          required
-   * @return the overlapping allocations
-   */
   public RLESparseResourceAllocation getRangeOverlapping(long start, long end) {
     readLock.lock();
     try {
       NavigableMap<Long, Resource> a = this.getCumulative();
+
       if (a != null && !a.isEmpty()) {
         // include the portion of previous entry that overlaps start
         if (start > a.firstKey()) {
           long previous = a.floorKey(start);
           a = a.tailMap(previous, true);
         }
+
         if (end < a.lastKey()) {
           a = a.headMap(end, true);
         }
+
       }
       RLESparseResourceAllocation ret =
           new RLESparseResourceAllocation(a, resourceCalculator);
@@ -505,33 +527,7 @@ public class RLESparseResourceAllocation {
     } finally {
       readLock.unlock();
     }
-  }
 
-  /**
-   * This method shifts all the timestamp of the {@link Resource} entries by the
-   * specified "delta".
-   *
-   * @param delta the time by which to shift the {@link Resource} allocations
-   */
-  public void shift(long delta) {
-    writeLock.lock();
-    try {
-      TreeMap<Long, Resource> newCum = new TreeMap<>();
-      long start;
-      for (Map.Entry<Long, Resource> entry : cumulativeCapacity.entrySet()) {
-        if (delta > 0) {
-          start = (entry.getKey() == Long.MAX_VALUE) ? Long.MAX_VALUE
-              : entry.getKey() + delta;
-        } else {
-          start = (entry.getKey() == Long.MIN_VALUE) ? Long.MIN_VALUE
-              : entry.getKey() + delta;
-        }
-        newCum.put(start, entry.getValue());
-      }
-      cumulativeCapacity = newCum;
-    } finally {
-      writeLock.unlock();
-    }
   }
 
   /**
@@ -545,8 +541,8 @@ public class RLESparseResourceAllocation {
   /**
    * Get the maximum capacity across specified time instances. The search-space
    * is specified using the starting value, tick, and the periodic interval for
-   * search. Maximum resource allocation across tick, tick + period, tick + 2 *
-   * period,..., tick + n * period .. is returned.
+   * search. Maximum resource allocation across tick, tick + period,
+   * tick + 2 * period,..., tick + n * period .. is returned.
    *
    * @param tick the starting time instance
    * @param period interval at which capacity is evaluated
@@ -554,19 +550,14 @@ public class RLESparseResourceAllocation {
    */
   public Resource getMaximumPeriodicCapacity(long tick, long period) {
     Resource maxCapacity = ZERO_RESOURCE;
-    readLock.lock();
-    try {
-      if (!cumulativeCapacity.isEmpty()) {
-        Long lastKey = cumulativeCapacity.lastKey();
-        for (long t = tick; t <= lastKey; t = t + period) {
-          maxCapacity = Resources.componentwiseMax(maxCapacity,
-              cumulativeCapacity.floorEntry(t).getValue());
-        }
+    if (!cumulativeCapacity.isEmpty()) {
+      Long lastKey = cumulativeCapacity.lastKey();
+      for (long t = tick; t <= lastKey; t = t + period) {
+        maxCapacity = Resources.componentwiseMax(maxCapacity,
+            cumulativeCapacity.floorEntry(t).getValue());
       }
-      return maxCapacity;
-    } finally {
-      readLock.unlock();
     }
+    return maxCapacity;
   }
 
   /**
@@ -576,17 +567,17 @@ public class RLESparseResourceAllocation {
    * @return minimum resource allocation
    */
   public Resource getMinimumCapacityInInterval(ReservationInterval interval) {
-    Resource minCapacity =
-        Resource.newInstance(Integer.MAX_VALUE, Integer.MAX_VALUE);
+    Resource minCapacity = Resource.newInstance(
+        Integer.MAX_VALUE, Integer.MAX_VALUE);
     long start = interval.getStartTime();
     long end = interval.getEndTime();
     NavigableMap<Long, Resource> capacityRange =
-        getRangeOverlapping(start, end).getCumulative();
+        this.getRangeOverlapping(start, end).getCumulative();
     if (!capacityRange.isEmpty()) {
       for (Map.Entry<Long, Resource> entry : capacityRange.entrySet()) {
         if (entry.getValue() != null) {
-          minCapacity =
-              Resources.componentwiseMin(minCapacity, entry.getValue());
+          minCapacity = Resources.componentwiseMin(minCapacity,
+              entry.getValue());
         }
       }
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e3345e98/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/ReservationAllocation.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/reservation/ReservationAllocation.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/ReservationAllocation.java
index bb4a7fb..0da95ac 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/ReservationAllocation.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/ReservationAllocation.java
@@ -24,16 +24,14 @@ import org.apache.hadoop.yarn.api.records.ReservationDefinition;
 import org.apache.hadoop.yarn.api.records.ReservationId;
 import org.apache.hadoop.yarn.api.records.Resource;
 
-import com.google.common.annotations.VisibleForTesting;
-
 /**
  * A ReservationAllocation represents a concrete allocation of resources over
  * time that satisfy a certain {@link ReservationDefinition}. This is used
  * internally by a {@link Plan} to store information about how each of the
  * accepted {@link ReservationDefinition} have been allocated.
  */
-public interface ReservationAllocation
-    extends Comparable<ReservationAllocation> {
+public interface ReservationAllocation extends
+    Comparable<ReservationAllocation> {
 
   /**
    * Returns the unique identifier {@link ReservationId} that represents the
@@ -42,28 +40,28 @@ public interface ReservationAllocation
    * @return reservationId the unique identifier {@link ReservationId} that
    *         represents the reservation
    */
-  ReservationId getReservationId();
+  public ReservationId getReservationId();
 
   /**
    * Returns the original {@link ReservationDefinition} submitted by the client
    * 
    * @return the {@link ReservationDefinition} submitted by the client
    */
-  ReservationDefinition getReservationDefinition();
+  public ReservationDefinition getReservationDefinition();
 
   /**
    * Returns the time at which the reservation is activated.
    * 
    * @return the time at which the reservation is activated
    */
-  long getStartTime();
+  public long getStartTime();
 
   /**
    * Returns the time at which the reservation terminates.
    * 
    * @return the time at which the reservation terminates
    */
-  long getEndTime();
+  public long getEndTime();
 
   /**
    * Returns the map of resources requested against the time interval for which
@@ -72,28 +70,28 @@ public interface ReservationAllocation
    * @return the allocationRequests the map of resources requested against the
    *         time interval for which they were
    */
-  Map<ReservationInterval, Resource> getAllocationRequests();
+  public Map<ReservationInterval, Resource> getAllocationRequests();
 
   /**
    * Return a string identifying the plan to which the reservation belongs
    * 
    * @return the plan to which the reservation belongs
    */
-  String getPlanName();
+  public String getPlanName();
 
   /**
    * Returns the user who requested the reservation
    * 
    * @return the user who requested the reservation
    */
-  String getUser();
+  public String getUser();
 
   /**
    * Returns whether the reservation has gang semantics or not
    * 
    * @return true if there is a gang request, false otherwise
    */
-  boolean containsGangs();
+  public boolean containsGangs();
 
   /**
    * Sets the time at which the reservation was accepted by the system
@@ -101,14 +99,14 @@ public interface ReservationAllocation
    * @param acceptedAt the time at which the reservation was accepted by the
    *          system
    */
-  void setAcceptanceTimestamp(long acceptedAt);
+  public void setAcceptanceTimestamp(long acceptedAt);
 
   /**
    * Returns the time at which the reservation was accepted by the system
    * 
    * @return the time at which the reservation was accepted by the system
    */
-  long getAcceptanceTime();
+  public long getAcceptanceTime();
 
   /**
    * Returns the capacity represented by cumulative resources reserved by the
@@ -118,42 +116,12 @@ public interface ReservationAllocation
    *          requested
    * @return the resources reserved at the specified time
    */
-  Resource getResourcesAtTime(long tick);
-
-  /**
-   * Return a RLE representation of used resources.
-   *
-   * @return a RLE encoding of resources allocated over time.
-   */
-  RLESparseResourceAllocation getResourcesOverTime();
-
+  public Resource getResourcesAtTime(long tick);
 
   /**
    * Return a RLE representation of used resources.
-   *
-   * @param start start of the time interval.
-   * @param end end of the time interval.
    * @return a RLE encoding of resources allocated over time.
    */
-  RLESparseResourceAllocation getResourcesOverTime(long start, long end);
-
-  /**
-   * Get the periodicity of this reservation representing the time period of the
-   * periodic job. Period is represented in milliseconds for periodic jobs.
-   * Period is 0 for non-periodic jobs.
-   *
-   * @return periodicity of this reservation
-   */
-  long getPeriodicity();
-
-  /**
-   * Set the periodicity of this reservation representing the time period of the
-   * periodic job. Period is represented in milliseconds for periodic jobs.
-   * Period is 0 for non-periodic jobs.
-   *
-   * @param period periodicity of this reservation
-   */
-  @VisibleForTesting
-  void setPeriodicity(long period);
+  public RLESparseResourceAllocation getResourcesOverTime();
 
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e3345e98/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/ReservationInputValidator.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/reservation/ReservationInputValidator.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/ReservationInputValidator.java
index a66d222..027d066 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/ReservationInputValidator.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/ReservationInputValidator.java
@@ -44,8 +44,6 @@ public class ReservationInputValidator {
 
   /**
    * Utility class to validate reservation requests.
-   *
-   * @param clock the {@link Clock} to use
    */
   public ReservationInputValidator(Clock clock) {
     this.clock = clock;
@@ -55,21 +53,22 @@ public class ReservationInputValidator {
       ReservationId reservationId, String auditConstant) throws YarnException {
     // check if the reservation id is valid
     if (reservationId == null) {
-      String message = "Missing reservation id."
-          + " Please try again by specifying a reservation id.";
+      String message =
+          "Missing reservation id."
+              + " Please try again by specifying a reservation id.";
       RMAuditLogger.logFailure("UNKNOWN", auditConstant,
           "validate reservation input", "ClientRMService", message);
       throw RPCUtil.getRemoteException(message);
     }
     String queue = reservationSystem.getQueueForReservation(reservationId);
     String nullQueueErrorMessage =
-        "The specified reservation with ID: " + reservationId
-            + " is unknown. Please try again with a valid reservation.";
+            "The specified reservation with ID: " + reservationId
+                    + " is unknown. Please try again with a valid reservation.";
     String nullPlanErrorMessage = "The specified reservation: " + reservationId
-        + " is not associated with any valid plan."
-        + " Please try again with a valid reservation.";
+                            + " is not associated with any valid plan."
+                            + " Please try again with a valid reservation.";
     return getPlanFromQueue(reservationSystem, queue, auditConstant,
-        nullQueueErrorMessage, nullPlanErrorMessage);
+            nullQueueErrorMessage, nullPlanErrorMessage);
   }
 
   private void validateReservationDefinition(ReservationId reservationId,
@@ -78,15 +77,17 @@ public class ReservationInputValidator {
     String message = "";
     // check if deadline is in the past
     if (contract == null) {
-      message = "Missing reservation definition."
-          + " Please try again by specifying a reservation definition.";
+      message =
+          "Missing reservation definition."
+              + " Please try again by specifying a reservation definition.";
       RMAuditLogger.logFailure("UNKNOWN", auditConstant,
           "validate reservation input definition", "ClientRMService", message);
       throw RPCUtil.getRemoteException(message);
     }
     if (contract.getDeadline() <= clock.getTime()) {
-      message = "The specified deadline: " + contract.getDeadline()
-          + " is the past. Please try again with deadline in the future.";
+      message =
+          "The specified deadline: " + contract.getDeadline()
+              + " is the past. Please try again with deadline in the future.";
       RMAuditLogger.logFailure("UNKNOWN", auditConstant,
           "validate reservation input definition", "ClientRMService", message);
       throw RPCUtil.getRemoteException(message);
@@ -94,16 +95,18 @@ public class ReservationInputValidator {
     // Check if at least one RR has been specified
     ReservationRequests resReqs = contract.getReservationRequests();
     if (resReqs == null) {
-      message = "No resources have been specified to reserve."
-          + "Please try again by specifying the resources to reserve.";
+      message =
+          "No resources have been specified to reserve."
+              + "Please try again by specifying the resources to reserve.";
       RMAuditLogger.logFailure("UNKNOWN", auditConstant,
           "validate reservation input definition", "ClientRMService", message);
       throw RPCUtil.getRemoteException(message);
     }
     List<ReservationRequest> resReq = resReqs.getReservationResources();
     if (resReq == null || resReq.isEmpty()) {
-      message = "No resources have been specified to reserve."
-          + " Please try again by specifying the resources to reserve.";
+      message =
+          "No resources have been specified to reserve."
+              + " Please try again by specifying the resources to reserve.";
       RMAuditLogger.logFailure("UNKNOWN", auditConstant,
           "validate reservation input definition", "ClientRMService", message);
       throw RPCUtil.getRemoteException(message);
@@ -120,18 +123,22 @@ public class ReservationInputValidator {
       } else {
         minDuration += rr.getDuration();
       }
-      maxGangSize = Resources.max(plan.getResourceCalculator(),
-          plan.getTotalCapacity(), maxGangSize,
-          Resources.multiply(rr.getCapability(), rr.getConcurrency()));
+      maxGangSize =
+          Resources.max(plan.getResourceCalculator(), plan.getTotalCapacity(),
+              maxGangSize,
+              Resources.multiply(rr.getCapability(), rr.getConcurrency()));
     }
     // verify the allocation is possible (skip for ANY)
     long duration = contract.getDeadline() - contract.getArrival();
-    if (duration < minDuration && type != ReservationRequestInterpreter.R_ANY) {
-      message = "The time difference (" + (duration) + ") between arrival ("
-          + contract.getArrival() + ") " + "and deadline ("
-          + contract.getDeadline() + ") must "
-          + " be greater or equal to the minimum resource duration ("
-          + minDuration + ")";
+    if (duration < minDuration
+        && type != ReservationRequestInterpreter.R_ANY) {
+      message =
+          "The time difference ("
+              + (duration)
+              + ") between arrival (" + contract.getArrival() + ") "
+              + "and deadline (" + contract.getDeadline() + ") must "
+              + " be greater or equal to the minimum resource duration ("
+              + minDuration + ")";
       RMAuditLogger.logFailure("UNKNOWN", auditConstant,
           "validate reservation input definition", "ClientRMService", message);
       throw RPCUtil.getRemoteException(message);
@@ -141,9 +148,10 @@ public class ReservationInputValidator {
     if (Resources.greaterThan(plan.getResourceCalculator(),
         plan.getTotalCapacity(), maxGangSize, plan.getTotalCapacity())
         && type != ReservationRequestInterpreter.R_ANY) {
-      message = "The size of the largest gang in the reservation definition ("
-          + maxGangSize + ") exceed the capacity available ("
-          + plan.getTotalCapacity() + " )";
+      message =
+          "The size of the largest gang in the reservation definition ("
+              + maxGangSize + ") exceed the capacity available ("
+              + plan.getTotalCapacity() + " )";
       RMAuditLogger.logFailure("UNKNOWN", auditConstant,
           "validate reservation input definition", "ClientRMService", message);
       throw RPCUtil.getRemoteException(message);
@@ -171,32 +179,32 @@ public class ReservationInputValidator {
     }
   }
 
-  private Plan getPlanFromQueue(ReservationSystem reservationSystem,
-      String queue, String auditConstant) throws YarnException {
+  private Plan getPlanFromQueue(ReservationSystem reservationSystem, String
+          queue, String auditConstant) throws YarnException {
     String nullQueueErrorMessage = "The queue is not specified."
-        + " Please try again with a valid reservable queue.";
+            + " Please try again with a valid reservable queue.";
     String nullPlanErrorMessage = "The specified queue: " + queue
-        + " is not managed by reservation system."
-        + " Please try again with a valid reservable queue.";
+            + " is not managed by reservation system."
+            + " Please try again with a valid reservable queue.";
     return getPlanFromQueue(reservationSystem, queue, auditConstant,
-        nullQueueErrorMessage, nullPlanErrorMessage);
+            nullQueueErrorMessage, nullPlanErrorMessage);
   }
 
-  private Plan getPlanFromQueue(ReservationSystem reservationSystem,
-      String queue, String auditConstant, String nullQueueErrorMessage,
-      String nullPlanErrorMessage) throws YarnException {
+  private Plan getPlanFromQueue(ReservationSystem reservationSystem, String
+          queue, String auditConstant, String nullQueueErrorMessage,
+          String nullPlanErrorMessage) throws YarnException {
     if (queue == null || queue.isEmpty()) {
       RMAuditLogger.logFailure("UNKNOWN", auditConstant,
-          "validate reservation input", "ClientRMService",
-          nullQueueErrorMessage);
+              "validate reservation input", "ClientRMService",
+              nullQueueErrorMessage);
       throw RPCUtil.getRemoteException(nullQueueErrorMessage);
     }
     // check if the associated plan is valid
     Plan plan = reservationSystem.getPlan(queue);
     if (plan == null) {
       RMAuditLogger.logFailure("UNKNOWN", auditConstant,
-          "validate reservation input", "ClientRMService",
-          nullPlanErrorMessage);
+              "validate reservation input", "ClientRMService",
+              nullPlanErrorMessage);
       throw RPCUtil.getRemoteException(nullPlanErrorMessage);
     }
     return plan;
@@ -214,21 +222,22 @@ public class ReservationInputValidator {
    * @param reservationId the {@link ReservationId} associated with the current
    *          request
    * @return the {@link Plan} to submit the request to
-   * @throws YarnException if validation fails
+   * @throws YarnException
    */
   public Plan validateReservationSubmissionRequest(
-      ReservationSystem reservationSystem, ReservationSubmissionRequest request,
-      ReservationId reservationId) throws YarnException {
+      ReservationSystem reservationSystem,
+      ReservationSubmissionRequest request, ReservationId reservationId)
+      throws YarnException {
     String message;
     if (reservationId == null) {
-      message = "Reservation id cannot be null. Please try again specifying "
-          + " a valid reservation id by creating a new reservation id.";
+      message = "Reservation id cannot be null. Please try again " +
+        "specifying a valid reservation id by creating a new reservation id.";
       throw RPCUtil.getRemoteException(message);
     }
     // Check if it is a managed queue
     String queue = request.getQueue();
     Plan plan = getPlanFromQueue(reservationSystem, queue,
-        AuditConstants.SUBMIT_RESERVATION_REQUEST);
+            AuditConstants.SUBMIT_RESERVATION_REQUEST);
 
     validateReservationDefinition(reservationId,
         request.getReservationDefinition(), plan,
@@ -246,14 +255,15 @@ public class ReservationInputValidator {
    * @param request the {@link ReservationUpdateRequest} defining the resources
    *          required over time for the request
    * @return the {@link Plan} to submit the request to
-   * @throws YarnException if validation fails
+   * @throws YarnException
    */
   public Plan validateReservationUpdateRequest(
       ReservationSystem reservationSystem, ReservationUpdateRequest request)
       throws YarnException {
     ReservationId reservationId = request.getReservationId();
-    Plan plan = validateReservation(reservationSystem, reservationId,
-        AuditConstants.UPDATE_RESERVATION_REQUEST);
+    Plan plan =
+        validateReservation(reservationSystem, reservationId,
+            AuditConstants.UPDATE_RESERVATION_REQUEST);
     validateReservationDefinition(reservationId,
         request.getReservationDefinition(), plan,
         AuditConstants.UPDATE_RESERVATION_REQUEST);
@@ -268,26 +278,28 @@ public class ReservationInputValidator {
    *
    * @param reservationSystem the {@link ReservationSystem} to validate against
    * @param request the {@link ReservationListRequest} defining search
-   *          parameters for reservations in the {@link ReservationSystem} that
-   *          is being validated against.
+   *                parameters for reservations in the {@link ReservationSystem}
+   *                that is being validated against.
    * @return the {@link Plan} to list reservations of.
-   * @throws YarnException if validation fails
+   * @throws YarnException
    */
   public Plan validateReservationListRequest(
-      ReservationSystem reservationSystem, ReservationListRequest request)
+      ReservationSystem reservationSystem,
+      ReservationListRequest request)
       throws YarnException {
     String queue = request.getQueue();
     if (request.getEndTime() < request.getStartTime()) {
-      String errorMessage = "The specified end time must be greater than "
-          + "the specified start time.";
+      String errorMessage = "The specified end time must be greater than " +
+              "the specified start time.";
       RMAuditLogger.logFailure("UNKNOWN",
-          AuditConstants.LIST_RESERVATION_REQUEST,
-          "validate list reservation input", "ClientRMService", errorMessage);
+              AuditConstants.LIST_RESERVATION_REQUEST,
+              "validate list reservation input", "ClientRMService",
+              errorMessage);
       throw RPCUtil.getRemoteException(errorMessage);
     }
     // Check if it is a managed queue
     return getPlanFromQueue(reservationSystem, queue,
-        AuditConstants.LIST_RESERVATION_REQUEST);
+            AuditConstants.LIST_RESERVATION_REQUEST);
   }
 
   /**
@@ -300,7 +312,7 @@ public class ReservationInputValidator {
    * @param request the {@link ReservationDeleteRequest} defining the resources
    *          required over time for the request
    * @return the {@link Plan} to submit the request to
-   * @throws YarnException if validation fails
+   * @throws YarnException
    */
   public Plan validateReservationDeleteRequest(
       ReservationSystem reservationSystem, ReservationDeleteRequest request)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e3345e98/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/ReservationSystem.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/reservation/ReservationSystem.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/ReservationSystem.java
index a6c8fcf..8b62972 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/ReservationSystem.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/ReservationSystem.java
@@ -18,8 +18,6 @@
 
 package org.apache.hadoop.yarn.server.resourcemanager.reservation;
 
-import java.util.Map;
-
 import org.apache.hadoop.classification.InterfaceAudience.LimitedPrivate;
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.conf.Configuration;
@@ -31,6 +29,8 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Queue;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.security.ReservationsACLsManager;
 
+import java.util.Map;
+
 /**
  * This interface is the one implemented by any system that wants to support
  * Reservations i.e. make {@code Resource} allocations in future. Implementors
@@ -57,7 +57,7 @@ public interface ReservationSystem extends Recoverable {
    * 
    * @param conf configuration
    * @param rmContext current context of the {@code ResourceManager}
-   * @throws YarnException if initialization of the configured plan fails
+   * @throws YarnException
    */
   void reinitialize(Configuration conf, RMContext rmContext)
       throws YarnException;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e3345e98/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/SharingPolicy.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/reservation/SharingPolicy.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/SharingPolicy.java
index cbf0f38..e458055 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/SharingPolicy.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/SharingPolicy.java
@@ -38,7 +38,7 @@ public interface SharingPolicy {
    * @param planQueuePath the name of the queue for this plan
    * @param conf the system configuration
    */
-  void init(String planQueuePath, ReservationSchedulerConfiguration conf);
+  public void init(String planQueuePath, ReservationSchedulerConfiguration conf);
 
   /**
    * This method runs the policy validation logic, and return true/false on
@@ -51,7 +51,7 @@ public interface SharingPolicy {
    * @throws PlanningException if the policy is respected if we add this
    *           {@link ReservationAllocation} to the {@link Plan}
    */
-  void validate(Plan plan, ReservationAllocation newAllocation)
+  public void validate(Plan plan, ReservationAllocation newAllocation)
       throws PlanningException;
 
   /**
@@ -68,13 +68,9 @@ public interface SharingPolicy {
    * @param start the start time for the range we are querying
    * @param end the end time for the range we are querying
    * @param oldId (optional) the id of a reservation being updated
-   *
-   * @return the available resources expressed as a
-   *         {@link RLESparseResourceAllocation}
-   *
    * @throws PlanningException throws if the request is not valid
    */
-  RLESparseResourceAllocation availableResources(
+  public RLESparseResourceAllocation availableResources(
       RLESparseResourceAllocation available, Plan plan, String user,
       ReservationId oldId, long start, long end) throws PlanningException;
 
@@ -86,6 +82,7 @@ public interface SharingPolicy {
    * 
    * @return validWindow the window of validity considered by the policy.
    */
-  long getValidWindow();
+  public long getValidWindow();
+
 
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e3345e98/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/planning/Planner.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/reservation/planning/Planner.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/planning/Planner.java
index af0e712..abac6ac 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/planning/Planner.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/planning/Planner.java
@@ -34,7 +34,7 @@ public interface Planner {
    *
    * @param plan the {@link Plan} to replan
    * @param contracts the list of reservation requests
-   * @throws PlanningException if operation is unsuccessful
+   * @throws PlanningException
    */
   public void plan(Plan plan, List<ReservationDefinition> contracts)
       throws PlanningException;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e3345e98/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/planning/PlanningAlgorithm.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/reservation/planning/PlanningAlgorithm.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/planning/PlanningAlgorithm.java
index bbbf0d6..199bfa5 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/planning/PlanningAlgorithm.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/planning/PlanningAlgorithm.java
@@ -50,7 +50,7 @@ public abstract class PlanningAlgorithm implements ReservationAgent {
    * @return whether the allocateUser function was successful or not
    *
    * @throws PlanningException if the session cannot be fitted into the plan
-   * @throws ContractValidationException if validation fails
+   * @throws ContractValidationException
    */
   protected boolean allocateUser(ReservationId reservationId, String user,
       Plan plan, ReservationDefinition contract,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e3345e98/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/planning/StageAllocator.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/reservation/planning/StageAllocator.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/planning/StageAllocator.java
index 8934b0f..ec6d9c0 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/planning/StageAllocator.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/planning/StageAllocator.java
@@ -50,7 +50,7 @@ public interface StageAllocator {
    *
    * @return The computed allocation (or null if the stage could not be
    *         allocated)
-   * @throws PlanningException if operation is unsuccessful
+   * @throws PlanningException
    */
   Map<ReservationInterval, Resource> computeStageAllocation(Plan plan,
       RLESparseResourceAllocation planLoads,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e3345e98/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/planning/StageAllocatorGreedy.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/reservation/planning/StageAllocatorGreedy.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/planning/StageAllocatorGreedy.java
index d107487..da04336 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/planning/StageAllocatorGreedy.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/planning/StageAllocatorGreedy.java
@@ -69,7 +69,7 @@ public class StageAllocatorGreedy implements StageAllocator {
 
     RLESparseResourceAllocation netAvailable =
         plan.getAvailableResourceOverTime(user, oldId, stageEarliestStart,
-            stageDeadline, 0);
+            stageDeadline);
 
     netAvailable =
         RLESparseResourceAllocation.merge(plan.getResourceCalculator(),

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e3345e98/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/planning/StageAllocatorGreedyRLE.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/reservation/planning/StageAllocatorGreedyRLE.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/planning/StageAllocatorGreedyRLE.java
index ae7d91a..ec83e02 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/planning/StageAllocatorGreedyRLE.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/planning/StageAllocatorGreedyRLE.java
@@ -83,8 +83,9 @@ public class StageAllocatorGreedyRLE implements StageAllocator {
     int gangsToPlace = rr.getNumContainers() / rr.getConcurrency();
 
     // get available resources from plan
-    RLESparseResourceAllocation netRLERes = plan.getAvailableResourceOverTime(
-        user, oldId, stageEarliestStart, stageDeadline, 0);
+    RLESparseResourceAllocation netRLERes =
+        plan.getAvailableResourceOverTime(user, oldId, stageEarliestStart,
+            stageDeadline);
 
     // remove plan modifications
     netRLERes =

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e3345e98/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/planning/StageAllocatorLowCostAligned.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/reservation/planning/StageAllocatorLowCostAligned.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/planning/StageAllocatorLowCostAligned.java
index c014549..e45f58c 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/planning/StageAllocatorLowCostAligned.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/planning/StageAllocatorLowCostAligned.java
@@ -77,8 +77,8 @@ public class StageAllocatorLowCostAligned implements StageAllocator {
     ResourceCalculator resCalc = plan.getResourceCalculator();
     Resource capacity = plan.getTotalCapacity();
 
-    RLESparseResourceAllocation netRLERes = plan.getAvailableResourceOverTime(
-        user, oldId, stageArrival, stageDeadline, 0);
+    RLESparseResourceAllocation netRLERes = plan
+        .getAvailableResourceOverTime(user, oldId, stageArrival, stageDeadline);
 
     long step = plan.getStep();
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e3345e98/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/ReservationSystemTestUtil.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/reservation/ReservationSystemTestUtil.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/ReservationSystemTestUtil.java
index 5337e06..e99842e 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/ReservationSystemTestUtil.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/ReservationSystemTestUtil.java
@@ -19,10 +19,7 @@ package org.apache.hadoop.yarn.server.resourcemanager.reservation;
 
 import static org.mockito.Matchers.any;
 import static org.mockito.Matchers.anySetOf;
-import static org.mockito.Mockito.doReturn;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.spy;
-import static org.mockito.Mockito.when;
+import static org.mockito.Mockito.*;
 
 import java.io.FileWriter;
 import java.io.IOException;
@@ -79,8 +76,7 @@ public class ReservationSystemTestUtil {
       String reservationQ, long timeWindow, float instConstraint,
       float avgConstraint) {
 
-    ReservationSchedulerConfiguration realConf =
-        new CapacitySchedulerConfiguration();
+    ReservationSchedulerConfiguration realConf = new CapacitySchedulerConfiguration();
     ReservationSchedulerConfiguration conf = spy(realConf);
     when(conf.getReservationWindow(reservationQ)).thenReturn(timeWindow);
     when(conf.getInstantaneousMaxCapacity(reservationQ))
@@ -172,6 +168,7 @@ public class ReservationSystemTestUtil {
     scheduler.start();
     scheduler.reinitialize(conf, rmContext);
 
+
     Resource resource =
         ReservationSystemTestUtil.calculateClusterResource(numContainers);
     RMNode node1 = MockNodes.newNodeInfo(1, resource, 1, "127.0.0.1");
@@ -187,16 +184,10 @@ public class ReservationSystemTestUtil {
 
   public static ReservationDefinition createSimpleReservationDefinition(
       long arrival, long deadline, long duration, int parallelism) {
-    return createSimpleReservationDefinition(arrival, deadline, duration,
-        parallelism, null);
-  }
-
-  public static ReservationDefinition createSimpleReservationDefinition(
-      long arrival, long deadline, long duration, int parallelism,
-      String recurrenceExpression) {
     // create a request with a single atomic ask
-    ReservationRequest r = ReservationRequest.newInstance(
-        Resource.newInstance(1024, 1), parallelism, parallelism, duration);
+    ReservationRequest r =
+        ReservationRequest.newInstance(Resource.newInstance(1024, 1),
+            parallelism, parallelism, duration);
     ReservationDefinition rDef = new ReservationDefinitionPBImpl();
     ReservationRequests reqs = new ReservationRequestsPBImpl();
     reqs.setReservationResources(Collections.singletonList(r));
@@ -204,31 +195,32 @@ public class ReservationSystemTestUtil {
     rDef.setReservationRequests(reqs);
     rDef.setArrival(arrival);
     rDef.setDeadline(deadline);
-    if (recurrenceExpression != null) {
-      rDef.setRecurrenceExpression(recurrenceExpression);
-    }
     return rDef;
   }
 
   public static ReservationSubmissionRequest createSimpleReservationRequest(
       ReservationId reservationId, int numContainers, long arrival,
       long deadline, long duration) {
-    return createSimpleReservationRequest(reservationId, numContainers, arrival,
-        deadline, duration, Priority.UNDEFINED);
+    return createSimpleReservationRequest(reservationId, numContainers,
+        arrival, deadline, duration, Priority.UNDEFINED);
   }
 
   public static ReservationSubmissionRequest createSimpleReservationRequest(
       ReservationId reservationId, int numContainers, long arrival,
       long deadline, long duration, Priority priority) {
     // create a request with a single atomic ask
-    ReservationRequest r = ReservationRequest
-        .newInstance(Resource.newInstance(1024, 1), numContainers, 1, duration);
-    ReservationRequests reqs = ReservationRequests.newInstance(
-        Collections.singletonList(r), ReservationRequestInterpreter.R_ALL);
-    ReservationDefinition rDef = ReservationDefinition.newInstance(arrival,
-        deadline, reqs, "testClientRMService#reservation", "0", priority);
-    ReservationSubmissionRequest request = ReservationSubmissionRequest
-        .newInstance(rDef, reservationQ, reservationId);
+    ReservationRequest r =
+        ReservationRequest.newInstance(Resource.newInstance(1024, 1),
+            numContainers, 1, duration);
+    ReservationRequests reqs =
+        ReservationRequests.newInstance(Collections.singletonList(r),
+            ReservationRequestInterpreter.R_ALL);
+    ReservationDefinition rDef =
+        ReservationDefinition.newInstance(arrival, deadline, reqs,
+            "testClientRMService#reservation", "0", priority);
+    ReservationSubmissionRequest request =
+        ReservationSubmissionRequest.newInstance(rDef,
+            reservationQ, reservationId);
     return request;
   }
 
@@ -260,9 +252,9 @@ public class ReservationSystemTestUtil {
     return cs;
   }
 
-  @SuppressWarnings("rawtypes")
-  public static void initializeRMContext(int numContainers,
-      AbstractYarnScheduler scheduler, RMContext mockRMContext) {
+  @SuppressWarnings("rawtypes") public static void initializeRMContext(
+      int numContainers, AbstractYarnScheduler scheduler,
+      RMContext mockRMContext) {
 
     when(mockRMContext.getScheduler()).thenReturn(scheduler);
     Resource r = calculateClusterResource(numContainers);
@@ -270,25 +262,26 @@ public class ReservationSystemTestUtil {
   }
 
   public static RMContext createRMContext(Configuration conf) {
-    RMContext mockRmContext = Mockito.spy(new RMContextImpl(null, null, null,
-        null, null, null, new RMContainerTokenSecretManager(conf),
-        new NMTokenSecretManagerInRM(conf),
-        new ClientToAMTokenSecretManagerInRM(), null));
+    RMContext mockRmContext = Mockito.spy(
+        new RMContextImpl(null, null, null, null, null, null,
+            new RMContainerTokenSecretManager(conf),
+            new NMTokenSecretManagerInRM(conf),
+            new ClientToAMTokenSecretManagerInRM(), null));
 
     RMNodeLabelsManager nlm = mock(RMNodeLabelsManager.class);
     when(nlm.getQueueResource(any(String.class), anySetOf(String.class),
-        any(Resource.class))).thenAnswer(new Answer<Resource>() {
-          @Override
-          public Resource answer(InvocationOnMock invocation) throws Throwable {
-            Object[] args = invocation.getArguments();
-            return (Resource) args[2];
-          }
-        });
+            any(Resource.class))).thenAnswer(new Answer<Resource>() {
+      @Override public Resource answer(InvocationOnMock invocation)
+          throws Throwable {
+        Object[] args = invocation.getArguments();
+        return (Resource) args[2];
+      }
+    });
 
     when(nlm.getResourceByLabel(any(String.class), any(Resource.class)))
         .thenAnswer(new Answer<Resource>() {
-          @Override
-          public Resource answer(InvocationOnMock invocation) throws Throwable {
+          @Override public Resource answer(InvocationOnMock invocation)
+              throws Throwable {
             Object[] args = invocation.getArguments();
             return (Resource) args[1];
           }
@@ -311,8 +304,9 @@ public class ReservationSystemTestUtil {
     final String A = CapacitySchedulerConfiguration.ROOT + ".a";
     conf.setCapacity(A, 10);
 
-    final String dedicated = CapacitySchedulerConfiguration.ROOT
-        + CapacitySchedulerConfiguration.DOT + reservationQ;
+    final String dedicated =
+        CapacitySchedulerConfiguration.ROOT + CapacitySchedulerConfiguration.DOT
+            + reservationQ;
     conf.setCapacity(dedicated, 80);
     // Set as reservation queue
     conf.setReservable(dedicated, true);
@@ -411,55 +405,26 @@ public class ReservationSystemTestUtil {
 
   public static Map<ReservationInterval, Resource> generateAllocation(
       long startTime, long step, int[] alloc) {
-    return generateAllocation(startTime, step, alloc, null);
-  }
-
-  public static Map<ReservationInterval, Resource> generateAllocation(
-      long startTime, long step, int[] alloc, String recurrenceExpression) {
     Map<ReservationInterval, Resource> req = new TreeMap<>();
-
-    long period = 0;
-    if (recurrenceExpression != null) {
-      period = Long.parseLong(recurrenceExpression);
-    }
-
-    long rStart;
-    long rEnd;
-    for (int j = 0; j < 86400000; j += period) {
-      for (int i = 0; i < alloc.length; i++) {
-        rStart = (startTime + i * step) + j * period;
-        rEnd = (startTime + (i + 1) * step) + j * period;
-        if (period > 0) {
-          rStart = rStart % period + j * period;
-          rEnd = rEnd % period + j * period;
-          if (rStart > rEnd) {
-            // skip wrap-around entry
-            continue;
-          }
-        }
-
-        req.put(new ReservationInterval(rStart, rEnd),
-            ReservationSystemUtil.toResource(ReservationRequest
-                .newInstance(Resource.newInstance(1024, 1), alloc[i])));
-
-      }
-      // execute only once if non-periodic
-      if (period == 0) {
-        break;
-      }
+    for (int i = 0; i < alloc.length; i++) {
+      req.put(new ReservationInterval(startTime + i * step,
+          startTime + (i + 1) * step), ReservationSystemUtil.toResource(
+          ReservationRequest
+              .newInstance(Resource.newInstance(1024, 1), alloc[i])));
     }
     return req;
   }
 
-  public static RLESparseResourceAllocation generateRLESparseResourceAllocation(
-      int[] alloc, long[] timeSteps) {
+  public static RLESparseResourceAllocation
+      generateRLESparseResourceAllocation(int[] alloc, long[] timeSteps) {
     TreeMap<Long, Resource> allocationsMap = new TreeMap<>();
     for (int i = 0; i < alloc.length; i++) {
       allocationsMap.put(timeSteps[i],
           Resource.newInstance(alloc[i], alloc[i]));
     }
-    RLESparseResourceAllocation rleVector = new RLESparseResourceAllocation(
-        allocationsMap, new DefaultResourceCalculator());
+    RLESparseResourceAllocation rleVector =
+        new RLESparseResourceAllocation(allocationsMap,
+            new DefaultResourceCalculator());
     return rleVector;
   }
 


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[24/37] hadoop git commit: Revert "Plan/ResourceAllocation data structure enhancements required to support recurring reservations in ReservationSystem."

Posted by ae...@apache.org.
Revert "Plan/ResourceAllocation data structure enhancements required to support recurring reservations in ReservationSystem."

This reverts commit 7996eca7dcfaa1bdf970e32022274f2699bef8a1.


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

Branch: refs/heads/HDFS-7240
Commit: e3345e985bff93c6c74a76747e45376c6027f42c
Parents: 22de944
Author: Subru Krishnan <su...@apache.org>
Authored: Wed Sep 6 16:39:23 2017 -0700
Committer: Subru Krishnan <su...@apache.org>
Committed: Wed Sep 6 16:39:23 2017 -0700

----------------------------------------------------------------------
 .../hadoop/yarn/conf/YarnConfiguration.java     |   6 -
 .../yarn/conf/TestYarnConfigurationFields.java  |   4 +-
 .../reservation/AbstractReservationSystem.java  |  90 ++--
 .../AbstractSchedulerPlanFollower.java          | 183 ++++----
 .../reservation/InMemoryPlan.java               | 400 +++++------------
 .../InMemoryReservationAllocation.java          |  36 +-
 .../reservation/NoOverCommitPolicy.java         |   2 +-
 .../PeriodicRLESparseResourceAllocation.java    | 130 ++----
 .../resourcemanager/reservation/PlanEdit.java   |  24 +-
 .../resourcemanager/reservation/PlanView.java   |  94 ++--
 .../RLESparseResourceAllocation.java            | 115 +++--
 .../reservation/ReservationAllocation.java      |  60 +--
 .../reservation/ReservationInputValidator.java  | 134 +++---
 .../reservation/ReservationSystem.java          |   6 +-
 .../reservation/SharingPolicy.java              |  13 +-
 .../reservation/planning/Planner.java           |   2 +-
 .../reservation/planning/PlanningAlgorithm.java |   2 +-
 .../reservation/planning/StageAllocator.java    |   2 +-
 .../planning/StageAllocatorGreedy.java          |   2 +-
 .../planning/StageAllocatorGreedyRLE.java       |   5 +-
 .../planning/StageAllocatorLowCostAligned.java  |   4 +-
 .../reservation/ReservationSystemTestUtil.java  | 135 +++---
 .../reservation/TestInMemoryPlan.java           | 431 +++++++------------
 ...TestPeriodicRLESparseResourceAllocation.java | 109 ++---
 .../TestRLESparseResourceAllocation.java        | 122 +++---
 .../planning/TestSimpleCapacityReplanner.java   |   8 +-
 26 files changed, 777 insertions(+), 1342 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/e3345e98/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
index 27ca957..4944821 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
@@ -262,12 +262,6 @@ public class YarnConfiguration extends Configuration {
   public static final long DEFAULT_RM_RESERVATION_SYSTEM_PLAN_FOLLOWER_TIME_STEP =
       1000L;
 
-  /** The maximum periodicity for the Reservation System. */
-  public static final String RM_RESERVATION_SYSTEM_MAX_PERIODICITY =
-      RM_PREFIX + "reservation-system.max-periodicity";
-  public static final long DEFAULT_RM_RESERVATION_SYSTEM_MAX_PERIODICITY =
-      86400000L;
-
   /**
    * Enable periodic monitor threads.
    * @see #RM_SCHEDULER_MONITOR_POLICIES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e3345e98/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/test/java/org/apache/hadoop/yarn/conf/TestYarnConfigurationFields.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/test/java/org/apache/hadoop/yarn/conf/TestYarnConfigurationFields.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/test/java/org/apache/hadoop/yarn/conf/TestYarnConfigurationFields.java
index 1d3111c..bd7bf93 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/test/java/org/apache/hadoop/yarn/conf/TestYarnConfigurationFields.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/test/java/org/apache/hadoop/yarn/conf/TestYarnConfigurationFields.java
@@ -33,7 +33,7 @@ import org.apache.hadoop.conf.TestConfigurationFieldsBase;
  */
 public class TestYarnConfigurationFields extends TestConfigurationFieldsBase {
 
-  @SuppressWarnings({"deprecation", "methodlength"})
+  @SuppressWarnings("deprecation")
   @Override
   public void initializeMemberVariables() {
     xmlFilename = new String("yarn-default.xml");
@@ -69,8 +69,6 @@ public class TestYarnConfigurationFields extends TestConfigurationFieldsBase {
     configurationPropsToSkipCompare.add(YarnConfiguration
         .YARN_SECURITY_SERVICE_AUTHORIZATION_COLLECTOR_NODEMANAGER_PROTOCOL);
     configurationPropsToSkipCompare.add(YarnConfiguration.CURATOR_LEADER_ELECTOR);
-    configurationPropsToSkipCompare
-        .add(YarnConfiguration.RM_RESERVATION_SYSTEM_MAX_PERIODICITY);
 
     // Federation default configs to be ignored
     configurationPropsToSkipCompare

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e3345e98/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/AbstractReservationSystem.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/reservation/AbstractReservationSystem.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/AbstractReservationSystem.java
index 5b8772c..5ef4912 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/AbstractReservationSystem.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/AbstractReservationSystem.java
@@ -18,17 +18,6 @@
 
 package org.apache.hadoop.yarn.server.resourcemanager.reservation;
 
-import java.util.HashMap;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.Set;
-import java.util.concurrent.ScheduledExecutorService;
-import java.util.concurrent.ScheduledThreadPoolExecutor;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicLong;
-import java.util.concurrent.locks.Lock;
-import java.util.concurrent.locks.ReentrantReadWriteLock;
-
 import org.apache.hadoop.classification.InterfaceAudience.LimitedPrivate;
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.conf.Configuration;
@@ -57,6 +46,17 @@ import org.apache.hadoop.yarn.util.resource.ResourceCalculator;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Set;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+
 /**
  * This is the implementation of {@link ReservationSystem} based on the
  * {@link ResourceScheduler}
@@ -66,8 +66,8 @@ import org.slf4j.LoggerFactory;
 public abstract class AbstractReservationSystem extends AbstractService
     implements ReservationSystem {
 
-  private static final Logger LOG =
-      LoggerFactory.getLogger(AbstractReservationSystem.class);
+  private static final Logger LOG = LoggerFactory
+      .getLogger(AbstractReservationSystem.class);
 
   // private static final String DEFAULT_CAPACITY_SCHEDULER_PLAN
 
@@ -103,8 +103,6 @@ public abstract class AbstractReservationSystem extends AbstractService
 
   private boolean isRecoveryEnabled = false;
 
-  private long maxPeriodicity;
-
   /**
    * Construct the service.
    * 
@@ -145,41 +143,36 @@ public abstract class AbstractReservationSystem extends AbstractService
     this.conf = conf;
     scheduler = rmContext.getScheduler();
     // Get the plan step size
-    planStepSize = conf.getTimeDuration(
-        YarnConfiguration.RM_RESERVATION_SYSTEM_PLAN_FOLLOWER_TIME_STEP,
-        YarnConfiguration.DEFAULT_RM_RESERVATION_SYSTEM_PLAN_FOLLOWER_TIME_STEP,
-        TimeUnit.MILLISECONDS);
+    planStepSize =
+        conf.getTimeDuration(
+            YarnConfiguration.RM_RESERVATION_SYSTEM_PLAN_FOLLOWER_TIME_STEP,
+            YarnConfiguration.DEFAULT_RM_RESERVATION_SYSTEM_PLAN_FOLLOWER_TIME_STEP,
+            TimeUnit.MILLISECONDS);
     if (planStepSize < 0) {
       planStepSize =
           YarnConfiguration.DEFAULT_RM_RESERVATION_SYSTEM_PLAN_FOLLOWER_TIME_STEP;
     }
-    maxPeriodicity =
-        conf.getLong(YarnConfiguration.RM_RESERVATION_SYSTEM_MAX_PERIODICITY,
-            YarnConfiguration.DEFAULT_RM_RESERVATION_SYSTEM_MAX_PERIODICITY);
-    if (maxPeriodicity <= 0) {
-      maxPeriodicity =
-          YarnConfiguration.DEFAULT_RM_RESERVATION_SYSTEM_MAX_PERIODICITY;
-    }
     // Create a plan corresponding to every reservable queue
     Set<String> planQueueNames = scheduler.getPlanQueues();
     for (String planQueueName : planQueueNames) {
       Plan plan = initializePlan(planQueueName);
       plans.put(planQueueName, plan);
     }
-    isRecoveryEnabled = conf.getBoolean(YarnConfiguration.RECOVERY_ENABLED,
+    isRecoveryEnabled = conf.getBoolean(
+        YarnConfiguration.RECOVERY_ENABLED,
         YarnConfiguration.DEFAULT_RM_RECOVERY_ENABLED);
 
     if (conf.getBoolean(YarnConfiguration.YARN_RESERVATION_ACL_ENABLE,
-        YarnConfiguration.DEFAULT_YARN_RESERVATION_ACL_ENABLE)
-        && conf.getBoolean(YarnConfiguration.YARN_ACL_ENABLE,
-            YarnConfiguration.DEFAULT_YARN_ACL_ENABLE)) {
+            YarnConfiguration.DEFAULT_YARN_RESERVATION_ACL_ENABLE) &&
+                    conf.getBoolean(YarnConfiguration.YARN_ACL_ENABLE,
+                            YarnConfiguration.DEFAULT_YARN_ACL_ENABLE)) {
       reservationsACLsManager = new ReservationsACLsManager(scheduler, conf);
     }
   }
 
   private void loadPlan(String planName,
       Map<ReservationId, ReservationAllocationStateProto> reservations)
-      throws PlanningException {
+          throws PlanningException {
     Plan plan = plans.get(planName);
     Resource minAllocation = getMinAllocation();
     ResourceCalculator rescCalculator = getResourceCalculator();
@@ -255,8 +248,8 @@ public abstract class AbstractReservationSystem extends AbstractService
       Class<?> planFollowerPolicyClazz =
           conf.getClassByName(planFollowerPolicyClassName);
       if (PlanFollower.class.isAssignableFrom(planFollowerPolicyClazz)) {
-        return (PlanFollower) ReflectionUtils
-            .newInstance(planFollowerPolicyClazz, conf);
+        return (PlanFollower) ReflectionUtils.newInstance(
+            planFollowerPolicyClazz, conf);
       } else {
         throw new YarnRuntimeException("Class: " + planFollowerPolicyClassName
             + " not instance of " + PlanFollower.class.getCanonicalName());
@@ -264,8 +257,7 @@ public abstract class AbstractReservationSystem extends AbstractService
     } catch (ClassNotFoundException e) {
       throw new YarnRuntimeException(
           "Could not instantiate PlanFollowerPolicy: "
-              + planFollowerPolicyClassName,
-          e);
+              + planFollowerPolicyClassName, e);
     }
   }
 
@@ -379,8 +371,9 @@ public abstract class AbstractReservationSystem extends AbstractService
   public ReservationId getNewReservationId() {
     writeLock.lock();
     try {
-      ReservationId resId = ReservationId.newInstance(
-          ResourceManager.getClusterTimeStamp(), resCounter.incrementAndGet());
+      ReservationId resId =
+          ReservationId.newInstance(ResourceManager.getClusterTimeStamp(),
+              resCounter.incrementAndGet());
       LOG.info("Allocated new reservationId: " + resId);
       return resId;
     } finally {
@@ -397,11 +390,8 @@ public abstract class AbstractReservationSystem extends AbstractService
    * Get the default reservation system corresponding to the scheduler
    * 
    * @param scheduler the scheduler for which the reservation system is required
-   *
-   * @return the {@link ReservationSystem} based on the configured scheduler
    */
-  public static String getDefaultReservationSystem(
-      ResourceScheduler scheduler) {
+  public static String getDefaultReservationSystem(ResourceScheduler scheduler) {
     if (scheduler instanceof CapacityScheduler) {
       return CapacityReservationSystem.class.getName();
     } else if (scheduler instanceof FairScheduler) {
@@ -419,11 +409,12 @@ public abstract class AbstractReservationSystem extends AbstractService
     Resource maxAllocation = getMaxAllocation();
     ResourceCalculator rescCalc = getResourceCalculator();
     Resource totCap = getPlanQueueCapacity(planQueueName);
-    Plan plan = new InMemoryPlan(getRootQueueMetrics(), adPolicy,
-        getAgent(planQueuePath), totCap, planStepSize, rescCalc, minAllocation,
-        maxAllocation, planQueueName, getReplanner(planQueuePath),
-        getReservationSchedulerConfiguration().getMoveOnExpiry(planQueuePath),
-        maxPeriodicity, rmContext);
+    Plan plan =
+        new InMemoryPlan(getRootQueueMetrics(), adPolicy,
+            getAgent(planQueuePath), totCap, planStepSize, rescCalc,
+            minAllocation, maxAllocation, planQueueName,
+            getReplanner(planQueuePath), getReservationSchedulerConfiguration()
+            .getMoveOnExpiry(planQueuePath), rmContext);
     LOG.info("Initialized plan {} based on reservable queue {}",
         plan.toString(), planQueueName);
     return plan;
@@ -486,8 +477,8 @@ public abstract class AbstractReservationSystem extends AbstractService
       Class<?> admissionPolicyClazz =
           conf.getClassByName(admissionPolicyClassName);
       if (SharingPolicy.class.isAssignableFrom(admissionPolicyClazz)) {
-        return (SharingPolicy) ReflectionUtils.newInstance(admissionPolicyClazz,
-            conf);
+        return (SharingPolicy) ReflectionUtils.newInstance(
+            admissionPolicyClazz, conf);
       } else {
         throw new YarnRuntimeException("Class: " + admissionPolicyClassName
             + " not instance of " + SharingPolicy.class.getCanonicalName());
@@ -502,7 +493,8 @@ public abstract class AbstractReservationSystem extends AbstractService
     return this.reservationsACLsManager;
   }
 
-  protected abstract ReservationSchedulerConfiguration getReservationSchedulerConfiguration();
+  protected abstract ReservationSchedulerConfiguration
+      getReservationSchedulerConfiguration();
 
   protected abstract String getPlanQueuePath(String planQueueName);
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e3345e98/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/AbstractSchedulerPlanFollower.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/reservation/AbstractSchedulerPlanFollower.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/AbstractSchedulerPlanFollower.java
index 9b6a0b0..90357e3 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/AbstractSchedulerPlanFollower.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/AbstractSchedulerPlanFollower.java
@@ -18,14 +18,6 @@
 
 package org.apache.hadoop.yarn.server.resourcemanager.reservation;
 
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.Comparator;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Set;
-
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ReservationId;
 import org.apache.hadoop.yarn.api.records.Resource;
@@ -41,17 +33,24 @@ import org.apache.hadoop.yarn.util.resource.Resources;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+
 public abstract class AbstractSchedulerPlanFollower implements PlanFollower {
-  private static final Logger LOG =
-      LoggerFactory.getLogger(AbstractSchedulerPlanFollower.class);
+  private static final Logger LOG = LoggerFactory
+      .getLogger(AbstractSchedulerPlanFollower.class);
 
   protected Collection<Plan> plans = new ArrayList<Plan>();
   protected YarnScheduler scheduler;
   protected Clock clock;
 
   @Override
-  public void init(Clock clock, ResourceScheduler sched,
-      Collection<Plan> plans) {
+  public void init(Clock clock, ResourceScheduler sched, Collection<Plan> plans) {
     this.clock = clock;
     this.scheduler = sched;
     this.plans.addAll(plans);
@@ -72,7 +71,7 @@ public abstract class AbstractSchedulerPlanFollower implements PlanFollower {
 
   @Override
   public synchronized void synchronizePlan(Plan plan, boolean shouldReplan) {
-    String planQueueName = plan.getQueueName();
+     String planQueueName = plan.getQueueName();
     if (LOG.isDebugEnabled()) {
       LOG.debug("Running plan follower edit policy for plan: " + planQueueName);
     }
@@ -83,14 +82,12 @@ public abstract class AbstractSchedulerPlanFollower implements PlanFollower {
       now += step - (now % step);
     }
     Queue planQueue = getPlanQueue(planQueueName);
-    if (planQueue == null) {
-      return;
-    }
+    if (planQueue == null) return;
 
     // first we publish to the plan the current availability of resources
     Resource clusterResources = scheduler.getClusterResource();
-    Resource planResources =
-        getPlanResources(plan, planQueue, clusterResources);
+    Resource planResources = getPlanResources(plan, planQueue,
+        clusterResources);
     Set<ReservationAllocation> currentReservations =
         plan.getReservationsAtTime(now);
     Set<String> curReservationNames = new HashSet<String>();
@@ -98,11 +95,12 @@ public abstract class AbstractSchedulerPlanFollower implements PlanFollower {
     int numRes = getReservedResources(now, currentReservations,
         curReservationNames, reservedResources);
     // create the default reservation queue if it doesnt exist
-    String defReservationId = getReservationIdFromQueueName(planQueueName)
-        + ReservationConstants.DEFAULT_QUEUE_SUFFIX;
-    String defReservationQueue =
-        getReservationQueueName(planQueueName, defReservationId);
-    createDefaultReservationQueue(planQueueName, planQueue, defReservationId);
+    String defReservationId = getReservationIdFromQueueName(planQueueName) +
+        ReservationConstants.DEFAULT_QUEUE_SUFFIX;
+    String defReservationQueue = getReservationQueueName(planQueueName,
+        defReservationId);
+    createDefaultReservationQueue(planQueueName, planQueue,
+        defReservationId);
     curReservationNames.add(defReservationId);
     // if the resources dedicated to this plan has shrunk invoke replanner
     boolean shouldResize = false;
@@ -151,8 +149,10 @@ public abstract class AbstractSchedulerPlanFollower implements PlanFollower {
       // sort allocations from the one giving up the most resources, to the
       // one asking for the most avoid order-of-operation errors that
       // temporarily violate 100% capacity bound
-      List<ReservationAllocation> sortedAllocations = sortByDelta(
-          new ArrayList<ReservationAllocation>(currentReservations), now, plan);
+      List<ReservationAllocation> sortedAllocations =
+          sortByDelta(
+              new ArrayList<ReservationAllocation>(currentReservations), now,
+              plan);
       for (ReservationAllocation res : sortedAllocations) {
         String currResId = res.getReservationId().toString();
         if (curReservationNames.contains(currResId)) {
@@ -163,9 +163,10 @@ public abstract class AbstractSchedulerPlanFollower implements PlanFollower {
         if (planResources.getMemorySize() > 0
             && planResources.getVirtualCores() > 0) {
           if (shouldResize) {
-            capToAssign = calculateReservationToPlanProportion(
-                plan.getResourceCalculator(), planResources, reservedResources,
-                capToAssign);
+            capToAssign =
+                calculateReservationToPlanProportion(
+                    plan.getResourceCalculator(), planResources,
+                    reservedResources, capToAssign);
           }
           targetCapacity =
               calculateReservationToPlanRatio(plan.getResourceCalculator(),
@@ -184,8 +185,7 @@ public abstract class AbstractSchedulerPlanFollower implements PlanFollower {
           maxCapacity = targetCapacity;
         }
         try {
-          setQueueEntitlement(planQueueName, currResId, targetCapacity,
-              maxCapacity);
+          setQueueEntitlement(planQueueName, currResId, targetCapacity, maxCapacity);
         } catch (YarnException e) {
           LOG.warn("Exception while trying to size reservation for plan: {}",
               currResId, planQueueName, e);
@@ -196,10 +196,9 @@ public abstract class AbstractSchedulerPlanFollower implements PlanFollower {
     // compute the default queue capacity
     float defQCap = 1.0f - totalAssignedCapacity;
     if (LOG.isDebugEnabled()) {
-      LOG.debug(
-          "PlanFollowerEditPolicyTask: total Plan Capacity: {} "
-              + "currReservation: {} default-queue capacity: {}",
-          planResources, numRes, defQCap);
+      LOG.debug("PlanFollowerEditPolicyTask: total Plan Capacity: {} "
+          + "currReservation: {} default-queue capacity: {}", planResources,
+          numRes, defQCap);
     }
     // set the default queue to eat-up all remaining capacity
     try {
@@ -226,11 +225,12 @@ public abstract class AbstractSchedulerPlanFollower implements PlanFollower {
   }
 
   protected void setQueueEntitlement(String planQueueName, String currResId,
-      float targetCapacity, float maxCapacity) throws YarnException {
-    String reservationQueueName =
-        getReservationQueueName(planQueueName, currResId);
-    scheduler.setEntitlement(reservationQueueName,
-        new QueueEntitlement(targetCapacity, maxCapacity));
+      float targetCapacity,
+      float maxCapacity) throws YarnException {
+    String reservationQueueName = getReservationQueueName(planQueueName,
+        currResId);
+    scheduler.setEntitlement(reservationQueueName, new QueueEntitlement(
+        targetCapacity, maxCapacity));
   }
 
   // Schedulers have different ways of naming queues. See YARN-2773
@@ -244,21 +244,14 @@ public abstract class AbstractSchedulerPlanFollower implements PlanFollower {
    * Then move all apps in the set of queues to the parent plan queue's default
    * reservation queue if move is enabled. Finally cleanups the queue by killing
    * any apps (if move is disabled or move failed) and removing the queue
-   *
-   * @param planQueueName the name of {@code PlanQueue}
-   * @param shouldMove flag to indicate if any running apps should be moved or
-   *          killed
-   * @param toRemove the remnant apps to clean up
-   * @param defReservationQueue the default {@code ReservationQueue} of the
-   *          {@link Plan}
    */
-  protected void cleanupExpiredQueues(String planQueueName, boolean shouldMove,
-      Set<String> toRemove, String defReservationQueue) {
+  protected void cleanupExpiredQueues(String planQueueName,
+      boolean shouldMove, Set<String> toRemove, String defReservationQueue) {
     for (String expiredReservationId : toRemove) {
       try {
         // reduce entitlement to 0
-        String expiredReservation =
-            getReservationQueueName(planQueueName, expiredReservationId);
+        String expiredReservation = getReservationQueueName(planQueueName,
+            expiredReservationId);
         setQueueEntitlement(planQueueName, expiredReservation, 0.0f, 0.0f);
         if (shouldMove) {
           moveAppsInQueueSync(expiredReservation, defReservationQueue);
@@ -282,7 +275,7 @@ public abstract class AbstractSchedulerPlanFollower implements PlanFollower {
    * reservation queue in a synchronous fashion
    */
   private void moveAppsInQueueSync(String expiredReservation,
-      String defReservationQueue) {
+                                   String defReservationQueue) {
     List<ApplicationAttemptId> activeApps =
         scheduler.getAppsInQueue(expiredReservation);
     if (activeApps.isEmpty()) {
@@ -294,16 +287,16 @@ public abstract class AbstractSchedulerPlanFollower implements PlanFollower {
         scheduler.moveApplication(app.getApplicationId(), defReservationQueue);
       } catch (YarnException e) {
         LOG.warn(
-            "Encountered unexpected error during migration of application: {}"
-                + " from reservation: {}",
+            "Encountered unexpected error during migration of application: {}" +
+                " from reservation: {}",
             app, expiredReservation, e);
       }
     }
   }
 
-  protected int getReservedResources(long now,
-      Set<ReservationAllocation> currentReservations,
-      Set<String> curReservationNames, Resource reservedResources) {
+  protected int getReservedResources(long now, Set<ReservationAllocation>
+      currentReservations, Set<String> curReservationNames,
+                                     Resource reservedResources) {
     int numRes = 0;
     if (currentReservations != null) {
       numRes = currentReservations.size();
@@ -319,30 +312,23 @@ public abstract class AbstractSchedulerPlanFollower implements PlanFollower {
    * Sort in the order from the least new amount of resources asked (likely
    * negative) to the highest. This prevents "order-of-operation" errors related
    * to exceeding 100% capacity temporarily.
-   *
-   * @param currentReservations the currently active reservations
-   * @param now the current time
-   * @param plan the {@link Plan} that is being considered
-   *
-   * @return the sorted list of {@link ReservationAllocation}s
    */
   protected List<ReservationAllocation> sortByDelta(
       List<ReservationAllocation> currentReservations, long now, Plan plan) {
-    Collections.sort(currentReservations,
-        new ReservationAllocationComparator(now, this, plan));
+    Collections.sort(currentReservations, new ReservationAllocationComparator(
+        now, this, plan));
     return currentReservations;
   }
 
   /**
-   * Get queue associated with reservable queue named.
-   *
-   * @param planQueueName name of the reservable queue
+   * Get queue associated with reservable queue named
+   * @param planQueueName Name of the reservable queue
    * @return queue associated with the reservable queue
    */
   protected abstract Queue getPlanQueue(String planQueueName);
 
   /**
-   * Resizes reservations based on currently available resources.
+   * Resizes reservations based on currently available resources
    */
   private Resource calculateReservationToPlanProportion(
       ResourceCalculator rescCalculator, Resource availablePlanResources,
@@ -352,7 +338,7 @@ public abstract class AbstractSchedulerPlanFollower implements PlanFollower {
   }
 
   /**
-   * Calculates ratio of reservationResources to planResources.
+   * Calculates ratio of reservationResources to planResources
    */
   private float calculateReservationToPlanRatio(
       ResourceCalculator rescCalculator, Resource clusterResources,
@@ -362,7 +348,7 @@ public abstract class AbstractSchedulerPlanFollower implements PlanFollower {
   }
 
   /**
-   * Check if plan resources are less than expected reservation resources.
+   * Check if plan resources are less than expected reservation resources
    */
   private boolean arePlanResourcesLessThanReservations(
       ResourceCalculator rescCalculator, Resource clusterResources,
@@ -372,56 +358,38 @@ public abstract class AbstractSchedulerPlanFollower implements PlanFollower {
   }
 
   /**
-   * Get a list of reservation queues for this planQueue.
-   *
-   * @param planQueue the queue for the current {@link Plan}
-   *
-   * @return the queues corresponding to the reservations
+   * Get a list of reservation queues for this planQueue
    */
   protected abstract List<? extends Queue> getChildReservationQueues(
       Queue planQueue);
 
   /**
-   * Add a new reservation queue for reservation currResId for this planQueue.
+   * Add a new reservation queue for reservation currResId for this planQueue
    */
-  protected abstract void addReservationQueue(String planQueueName, Queue queue,
-      String currResId);
+  protected abstract void addReservationQueue(
+      String planQueueName, Queue queue, String currResId);
 
   /**
-   * Creates the default reservation queue for use when no reservation is used
-   * for applications submitted to this planQueue.
-   *
-   * @param planQueueName name of the reservable queue
-   * @param queue the queue for the current {@link Plan}
-   * @param defReservationQueue name of the default {@code ReservationQueue}
+   * Creates the default reservation queue for use when no reservation is
+   * used for applications submitted to this planQueue
    */
-  protected abstract void createDefaultReservationQueue(String planQueueName,
-      Queue queue, String defReservationQueue);
+  protected abstract void createDefaultReservationQueue(
+      String planQueueName, Queue queue, String defReservationQueue);
 
   /**
-   * Get plan resources for this planQueue.
-   *
-   * @param plan the current {@link Plan} being considered
-   * @param clusterResources the resources available in the cluster
-   *
-   * @return the resources allocated to the specified {@link Plan}
+   * Get plan resources for this planQueue
    */
-  protected abstract Resource getPlanResources(Plan plan, Queue queue,
-      Resource clusterResources);
+  protected abstract Resource getPlanResources(
+      Plan plan, Queue queue, Resource clusterResources);
 
   /**
    * Get reservation queue resources if it exists otherwise return null.
-   *
-   * @param plan the current {@link Plan} being considered
-   * @param reservationId the identifier of the reservation
-   *
-   * @return the resources allocated to the specified reservation
    */
   protected abstract Resource getReservationQueueResourceIfExists(Plan plan,
       ReservationId reservationId);
 
-  private static class ReservationAllocationComparator
-      implements Comparator<ReservationAllocation> {
+  private static class ReservationAllocationComparator implements
+      Comparator<ReservationAllocation> {
     AbstractSchedulerPlanFollower planFollower;
     long now;
     Plan plan;
@@ -436,12 +404,14 @@ public abstract class AbstractSchedulerPlanFollower implements PlanFollower {
     private Resource getUnallocatedReservedResources(
         ReservationAllocation reservation) {
       Resource resResource;
-      Resource reservationResource =
-          planFollower.getReservationQueueResourceIfExists(plan,
-              reservation.getReservationId());
+      Resource reservationResource = planFollower
+          .getReservationQueueResourceIfExists
+              (plan, reservation.getReservationId());
       if (reservationResource != null) {
-        resResource = Resources.subtract(reservation.getResourcesAtTime(now),
-            reservationResource);
+        resResource =
+            Resources.subtract(
+                reservation.getResourcesAtTime(now),
+                reservationResource);
       } else {
         resResource = reservation.getResourcesAtTime(now);
       }
@@ -458,3 +428,4 @@ public abstract class AbstractSchedulerPlanFollower implements PlanFollower {
     }
   }
 }
+

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e3345e98/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/InMemoryPlan.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/reservation/InMemoryPlan.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/InMemoryPlan.java
index 9eb1820..783fd09 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/InMemoryPlan.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/InMemoryPlan.java
@@ -6,9 +6,9 @@
  * 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>
+ *
+ *     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.
@@ -33,10 +33,9 @@ import java.util.concurrent.locks.ReentrantReadWriteLock;
 
 import org.apache.hadoop.yarn.api.records.ReservationId;
 import org.apache.hadoop.yarn.api.records.Resource;
-import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.server.resourcemanager.reservation.RLESparseResourceAllocation.RLEOperator;
 import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
 import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore;
-import org.apache.hadoop.yarn.server.resourcemanager.reservation.RLESparseResourceAllocation.RLEOperator;
 import org.apache.hadoop.yarn.server.resourcemanager.reservation.exceptions.PlanningException;
 import org.apache.hadoop.yarn.server.resourcemanager.reservation.planning.Planner;
 import org.apache.hadoop.yarn.server.resourcemanager.reservation.planning.ReservationAgent;
@@ -65,14 +64,9 @@ public class InMemoryPlan implements Plan {
 
   private RLESparseResourceAllocation rleSparseVector;
 
-  private PeriodicRLESparseResourceAllocation periodicRle;
-
   private Map<String, RLESparseResourceAllocation> userResourceAlloc =
       new HashMap<String, RLESparseResourceAllocation>();
 
-  private Map<String, RLESparseResourceAllocation> userPeriodicResourceAlloc =
-      new HashMap<String, RLESparseResourceAllocation>();
-
   private Map<String, RLESparseResourceAllocation> userActiveReservationCount =
       new HashMap<String, RLESparseResourceAllocation>();
 
@@ -102,27 +96,15 @@ public class InMemoryPlan implements Plan {
       String queueName, Planner replanner, boolean getMoveOnExpiry,
       RMContext rmContext) {
     this(queueMetrics, policy, agent, totalCapacity, step, resCalc, minAlloc,
-        maxAlloc, queueName, replanner, getMoveOnExpiry,
-        YarnConfiguration.DEFAULT_RM_RESERVATION_SYSTEM_MAX_PERIODICITY,
-        rmContext);
+        maxAlloc, queueName, replanner, getMoveOnExpiry, rmContext,
+        new UTCClock());
   }
 
   public InMemoryPlan(QueueMetrics queueMetrics, SharingPolicy policy,
       ReservationAgent agent, Resource totalCapacity, long step,
       ResourceCalculator resCalc, Resource minAlloc, Resource maxAlloc,
       String queueName, Planner replanner, boolean getMoveOnExpiry,
-      long maxPeriodicty, RMContext rmContext) {
-    this(queueMetrics, policy, agent, totalCapacity, step, resCalc, minAlloc,
-        maxAlloc, queueName, replanner, getMoveOnExpiry, maxPeriodicty,
-        rmContext, new UTCClock());
-  }
-
-  @SuppressWarnings("checkstyle:parameternumber")
-  public InMemoryPlan(QueueMetrics queueMetrics, SharingPolicy policy,
-      ReservationAgent agent, Resource totalCapacity, long step,
-      ResourceCalculator resCalc, Resource minAlloc, Resource maxAlloc,
-      String queueName, Planner replanner, boolean getMoveOnExpiry,
-      long maxPeriodicty, RMContext rmContext, Clock clock) {
+      RMContext rmContext, Clock clock) {
     this.queueMetrics = queueMetrics;
     this.policy = policy;
     this.agent = agent;
@@ -132,8 +114,6 @@ public class InMemoryPlan implements Plan {
     this.minAlloc = minAlloc;
     this.maxAlloc = maxAlloc;
     this.rleSparseVector = new RLESparseResourceAllocation(resCalc);
-    this.periodicRle =
-        new PeriodicRLESparseResourceAllocation(resCalc, maxPeriodicty);
     this.queueName = queueName;
     this.replanner = replanner;
     this.getMoveOnExpiry = getMoveOnExpiry;
@@ -146,39 +126,6 @@ public class InMemoryPlan implements Plan {
     return queueMetrics;
   }
 
-  private RLESparseResourceAllocation getUserRLEResourceAllocation(String user,
-      long period) {
-    RLESparseResourceAllocation resAlloc = null;
-    if (period > 0) {
-      if (userPeriodicResourceAlloc.containsKey(user)) {
-        resAlloc = userPeriodicResourceAlloc.get(user);
-      } else {
-        resAlloc = new PeriodicRLESparseResourceAllocation(resCalc,
-            periodicRle.getTimePeriod());
-        userPeriodicResourceAlloc.put(user, resAlloc);
-      }
-    } else {
-      if (userResourceAlloc.containsKey(user)) {
-        resAlloc = userResourceAlloc.get(user);
-      } else {
-        resAlloc = new RLESparseResourceAllocation(resCalc);
-        userResourceAlloc.put(user, resAlloc);
-      }
-    }
-    return resAlloc;
-  }
-
-  private void gcUserRLEResourceAllocation(String user, long period) {
-    if (period > 0) {
-      if (userPeriodicResourceAlloc.get(user).isEmpty()) {
-        userPeriodicResourceAlloc.remove(user);
-      }
-    } else {
-      if (userResourceAlloc.get(user).isEmpty()) {
-        userResourceAlloc.remove(user);
-      }
-    }
-  }
 
   private void incrementAllocation(ReservationAllocation reservation) {
     assert (readWriteLock.isWriteLockedByCurrentThread());
@@ -186,10 +133,11 @@ public class InMemoryPlan implements Plan {
         reservation.getAllocationRequests();
     // check if we have encountered the user earlier and if not add an entry
     String user = reservation.getUser();
-    long period = reservation.getPeriodicity();
-    RLESparseResourceAllocation resAlloc =
-        getUserRLEResourceAllocation(user, period);
-
+    RLESparseResourceAllocation resAlloc = userResourceAlloc.get(user);
+    if (resAlloc == null) {
+      resAlloc = new RLESparseResourceAllocation(resCalc);
+      userResourceAlloc.put(user, resAlloc);
+    }
     RLESparseResourceAllocation resCount = userActiveReservationCount.get(user);
     if (resCount == null) {
       resCount = new RLESparseResourceAllocation(resCalc);
@@ -201,43 +149,14 @@ public class InMemoryPlan implements Plan {
 
     for (Map.Entry<ReservationInterval, Resource> r : allocationRequests
         .entrySet()) {
-
-      if (period > 0L) {
-        for (int i = 0; i < periodicRle.getTimePeriod() / period; i++) {
-
-          long rStart = r.getKey().getStartTime() + i * period;
-          long rEnd = r.getKey().getEndTime() + i * period;
-
-          // handle wrap-around
-          if (rEnd > periodicRle.getTimePeriod()) {
-            long diff = rEnd - periodicRle.getTimePeriod();
-            rEnd = periodicRle.getTimePeriod();
-            ReservationInterval newInterval = new ReservationInterval(0, diff);
-            periodicRle.addInterval(newInterval, r.getValue());
-            resAlloc.addInterval(newInterval, r.getValue());
-          }
-
-          ReservationInterval newInterval =
-              new ReservationInterval(rStart, rEnd);
-          periodicRle.addInterval(newInterval, r.getValue());
-          resAlloc.addInterval(newInterval, r.getValue());
-        }
-
-      } else {
-        rleSparseVector.addInterval(r.getKey(), r.getValue());
-        resAlloc.addInterval(r.getKey(), r.getValue());
-        if (Resources.greaterThan(resCalc, totalCapacity, r.getValue(),
-            ZERO_RESOURCE)) {
-          earliestActive = Math.min(earliestActive, r.getKey().getStartTime());
-          latestActive = Math.max(latestActive, r.getKey().getEndTime());
-        }
+      resAlloc.addInterval(r.getKey(), r.getValue());
+      rleSparseVector.addInterval(r.getKey(), r.getValue());
+      if (Resources.greaterThan(resCalc, totalCapacity, r.getValue(),
+          ZERO_RESOURCE)) {
+        earliestActive = Math.min(earliestActive, r.getKey().getStartTime());
+        latestActive = Math.max(latestActive, r.getKey().getEndTime());
       }
     }
-    // periodic reservations are active from start time and good till cancelled
-    if (period > 0L) {
-      earliestActive = reservation.getStartTime();
-      latestActive = Long.MAX_VALUE;
-    }
     resCount.addInterval(new ReservationInterval(earliestActive, latestActive),
         Resource.newInstance(1, 1));
   }
@@ -247,55 +166,27 @@ public class InMemoryPlan implements Plan {
     Map<ReservationInterval, Resource> allocationRequests =
         reservation.getAllocationRequests();
     String user = reservation.getUser();
-    long period = reservation.getPeriodicity();
-    RLESparseResourceAllocation resAlloc =
-        getUserRLEResourceAllocation(user, period);
+    RLESparseResourceAllocation resAlloc = userResourceAlloc.get(user);
 
     long earliestActive = Long.MAX_VALUE;
     long latestActive = Long.MIN_VALUE;
     for (Map.Entry<ReservationInterval, Resource> r : allocationRequests
         .entrySet()) {
-      if (period > 0L) {
-        for (int i = 0; i < periodicRle.getTimePeriod() / period; i++) {
-
-          long rStart = r.getKey().getStartTime() + i * period;
-          long rEnd = r.getKey().getEndTime() + i * period;
-
-          // handle wrap-around
-          if (rEnd > periodicRle.getTimePeriod()) {
-            long diff = rEnd - periodicRle.getTimePeriod();
-            rEnd = periodicRle.getTimePeriod();
-            ReservationInterval newInterval = new ReservationInterval(0, diff);
-            periodicRle.removeInterval(newInterval, r.getValue());
-            resAlloc.removeInterval(newInterval, r.getValue());
-          }
-
-          ReservationInterval newInterval =
-              new ReservationInterval(rStart, rEnd);
-          periodicRle.removeInterval(newInterval, r.getValue());
-          resAlloc.removeInterval(newInterval, r.getValue());
-        }
-      } else {
-        rleSparseVector.removeInterval(r.getKey(), r.getValue());
-        resAlloc.removeInterval(r.getKey(), r.getValue());
-        if (Resources.greaterThan(resCalc, totalCapacity, r.getValue(),
-            ZERO_RESOURCE)) {
-          earliestActive = Math.min(earliestActive, r.getKey().getStartTime());
-          latestActive = Math.max(latestActive, r.getKey().getEndTime());
-        }
+      resAlloc.removeInterval(r.getKey(), r.getValue());
+      rleSparseVector.removeInterval(r.getKey(), r.getValue());
+      if (Resources.greaterThan(resCalc, totalCapacity, r.getValue(),
+          ZERO_RESOURCE)) {
+        earliestActive = Math.min(earliestActive, r.getKey().getStartTime());
+        latestActive = Math.max(latestActive, r.getKey().getEndTime());
       }
     }
-    gcUserRLEResourceAllocation(user, period);
+    if (resAlloc.isEmpty()) {
+      userResourceAlloc.remove(user);
+    }
 
     RLESparseResourceAllocation resCount = userActiveReservationCount.get(user);
-    // periodic reservations are active from start time and good till cancelled
-    if (period > 0L) {
-      earliestActive = reservation.getStartTime();
-      latestActive = Long.MAX_VALUE;
-    }
-    resCount.removeInterval(
-        new ReservationInterval(earliestActive, latestActive),
-        Resource.newInstance(1, 1));
+    resCount.removeInterval(new ReservationInterval(earliestActive,
+        latestActive), Resource.newInstance(1, 1));
     if (resCount.isEmpty()) {
       userActiveReservationCount.remove(user);
     }
@@ -307,9 +198,9 @@ public class InMemoryPlan implements Plan {
       if (currentReservations != null) {
         Set<ReservationAllocation> flattenedReservations =
             new TreeSet<ReservationAllocation>();
-        for (Set<InMemoryReservationAllocation> res : currentReservations
-            .values()) {
-          flattenedReservations.addAll(res);
+        for (Set<InMemoryReservationAllocation> reservationEntries :
+            currentReservations.values()) {
+          flattenedReservations.addAll(reservationEntries);
         }
         return flattenedReservations;
       } else {
@@ -327,16 +218,19 @@ public class InMemoryPlan implements Plan {
     InMemoryReservationAllocation inMemReservation =
         (InMemoryReservationAllocation) reservation;
     if (inMemReservation.getUser() == null) {
-      String errMsg = "The specified Reservation with ID "
-          + inMemReservation.getReservationId() + " is not mapped to any user";
+      String errMsg =
+          "The specified Reservation with ID "
+              + inMemReservation.getReservationId()
+              + " is not mapped to any user";
       LOG.error(errMsg);
       throw new IllegalArgumentException(errMsg);
     }
     writeLock.lock();
     try {
       if (reservationTable.containsKey(inMemReservation.getReservationId())) {
-        String errMsg = "The specified Reservation with ID "
-            + inMemReservation.getReservationId() + " already exists";
+        String errMsg =
+            "The specified Reservation with ID "
+                + inMemReservation.getReservationId() + " already exists";
         LOG.error(errMsg);
         throw new IllegalArgumentException(errMsg);
       }
@@ -352,8 +246,9 @@ public class InMemoryPlan implements Plan {
               getQueueName(), inMemReservation.getReservationId().toString());
         }
       }
-      ReservationInterval searchInterval = new ReservationInterval(
-          inMemReservation.getStartTime(), inMemReservation.getEndTime());
+      ReservationInterval searchInterval =
+          new ReservationInterval(inMemReservation.getStartTime(),
+              inMemReservation.getEndTime());
       Set<InMemoryReservationAllocation> reservations =
           currentReservations.get(searchInterval);
       if (reservations == null) {
@@ -385,8 +280,9 @@ public class InMemoryPlan implements Plan {
       ReservationId resId = reservation.getReservationId();
       ReservationAllocation currReservation = getReservationById(resId);
       if (currReservation == null) {
-        String errMsg = "The specified Reservation with ID " + resId
-            + " does not exist in the plan";
+        String errMsg =
+            "The specified Reservation with ID " + resId
+                + " does not exist in the plan";
         LOG.error(errMsg);
         throw new IllegalArgumentException(errMsg);
       }
@@ -422,8 +318,9 @@ public class InMemoryPlan implements Plan {
 
   private boolean removeReservation(ReservationAllocation reservation) {
     assert (readWriteLock.isWriteLockedByCurrentThread());
-    ReservationInterval searchInterval = new ReservationInterval(
-        reservation.getStartTime(), reservation.getEndTime());
+    ReservationInterval searchInterval =
+        new ReservationInterval(reservation.getStartTime(),
+            reservation.getEndTime());
     Set<InMemoryReservationAllocation> reservations =
         currentReservations.get(searchInterval);
     if (reservations != null) {
@@ -440,15 +337,16 @@ public class InMemoryPlan implements Plan {
         currentReservations.remove(searchInterval);
       }
     } else {
-      String errMsg = "The specified Reservation with ID "
-          + reservation.getReservationId() + " does not exist in the plan";
+      String errMsg =
+          "The specified Reservation with ID " + reservation.getReservationId()
+              + " does not exist in the plan";
       LOG.error(errMsg);
       throw new IllegalArgumentException(errMsg);
     }
     reservationTable.remove(reservation.getReservationId());
     decrementAllocation(reservation);
     LOG.info("Sucessfully deleted reservation: {} in plan.",
-        reservation.getReservationId());
+            reservation.getReservationId());
     return true;
   }
 
@@ -458,8 +356,9 @@ public class InMemoryPlan implements Plan {
     try {
       ReservationAllocation reservation = getReservationById(reservationID);
       if (reservation == null) {
-        String errMsg = "The specified Reservation with ID " + reservationID
-            + " does not exist in the plan";
+        String errMsg =
+            "The specified Reservation with ID " + reservationID
+                + " does not exist in the plan";
         LOG.error(errMsg);
         throw new IllegalArgumentException(errMsg);
       }
@@ -554,90 +453,66 @@ public class InMemoryPlan implements Plan {
       long start, long end) {
     readLock.lock();
     try {
-      // merge periodic and non-periodic allocations
       RLESparseResourceAllocation userResAlloc = userResourceAlloc.get(user);
-      RLESparseResourceAllocation userPeriodicResAlloc =
-          userPeriodicResourceAlloc.get(user);
 
-      if (userResAlloc != null && userPeriodicResAlloc != null) {
-        return RLESparseResourceAllocation.merge(resCalc, totalCapacity,
-            userResAlloc, userPeriodicResAlloc, RLEOperator.add, start, end);
-      }
       if (userResAlloc != null) {
         return userResAlloc.getRangeOverlapping(start, end);
+      } else {
+        return new RLESparseResourceAllocation(resCalc);
       }
-      if (userPeriodicResAlloc != null) {
-        return userPeriodicResAlloc.getRangeOverlapping(start, end);
-      }
-    } catch (PlanningException e) {
-      LOG.warn("Exception while trying to merge periodic"
-          + " and non-periodic user allocations: {}", e.getMessage(), e);
     } finally {
       readLock.unlock();
     }
-    return new RLESparseResourceAllocation(resCalc);
   }
 
   @Override
   public Resource getTotalCommittedResources(long t) {
     readLock.lock();
     try {
-      return Resources.add(rleSparseVector.getCapacityAtTime(t),
-          periodicRle.getCapacityAtTime(t));
+      return rleSparseVector.getCapacityAtTime(t);
     } finally {
       readLock.unlock();
     }
   }
 
   @Override
-  public Set<ReservationAllocation> getReservations(ReservationId reservationID,
-      ReservationInterval interval) {
+  public Set<ReservationAllocation> getReservations(ReservationId
+                    reservationID, ReservationInterval interval) {
     return getReservations(reservationID, interval, null);
   }
 
   @Override
-  public Set<ReservationAllocation> getReservations(ReservationId reservationID,
-      ReservationInterval interval, String user) {
+  public Set<ReservationAllocation> getReservations(ReservationId
+                    reservationID, ReservationInterval interval, String user) {
     if (reservationID != null) {
       ReservationAllocation allocation = getReservationById(reservationID);
-      if (allocation == null) {
+      if (allocation == null){
         return Collections.emptySet();
       }
       return Collections.singleton(allocation);
     }
 
-    long startTime = interval == null ? 0 : interval.getStartTime();
-    long endTime = interval == null ? Long.MAX_VALUE : interval.getEndTime();
+    long startTime = interval == null? 0 : interval.getStartTime();
+    long endTime = interval == null? Long.MAX_VALUE : interval.getEndTime();
 
     ReservationInterval searchInterval =
-        new ReservationInterval(endTime, Long.MAX_VALUE);
+            new ReservationInterval(endTime, Long.MAX_VALUE);
     readLock.lock();
     try {
-      SortedMap<ReservationInterval, Set<InMemoryReservationAllocation>> res =
-          currentReservations.headMap(searchInterval, true);
-      if (!res.isEmpty()) {
-        Set<ReservationAllocation> flattenedReservations = new HashSet<>();
-        for (Set<InMemoryReservationAllocation> resEntries : res.values()) {
-          for (InMemoryReservationAllocation reservation : resEntries) {
-            // validate user
-            if (user != null && !user.isEmpty()
-                && !reservation.getUser().equals(user)) {
-              continue;
-            }
-            // handle periodic reservations
-            long period = reservation.getPeriodicity();
-            if (period > 0) {
-              long t = endTime % period;
-              // check for both contained and wrap-around reservations
-              if ((t - startTime) * (t - endTime)
-                  * (startTime - endTime) >= 0) {
-                flattenedReservations.add(reservation);
-              }
-            } else {
-              // check for non-periodic reservations
-              if (reservation.getEndTime() > startTime) {
-                flattenedReservations.add(reservation);
+      SortedMap<ReservationInterval, Set<InMemoryReservationAllocation>>
+            reservations = currentReservations.headMap(searchInterval, true);
+      if (!reservations.isEmpty()) {
+        Set<ReservationAllocation> flattenedReservations =
+                new HashSet<>();
+        for (Set<InMemoryReservationAllocation> reservationEntries :
+                reservations.values()) {
+          for (InMemoryReservationAllocation res : reservationEntries) {
+            if (res.getEndTime() > startTime) {
+              if (user != null && !user.isEmpty()
+                      && !res.getUser().equals(user)) {
+                continue;
               }
+              flattenedReservations.add(res);
             }
           }
         }
@@ -675,82 +550,36 @@ public class InMemoryPlan implements Plan {
 
   @Override
   public RLESparseResourceAllocation getAvailableResourceOverTime(String user,
-      ReservationId oldId, long start, long end, long period)
-      throws PlanningException {
+      ReservationId oldId, long start, long end) throws PlanningException {
     readLock.lock();
     try {
-
-      // for non-periodic return simple available resources
-      if (period == 0) {
-
-        // create RLE of totCapacity
-        TreeMap<Long, Resource> totAvailable = new TreeMap<Long, Resource>();
-        totAvailable.put(start, Resources.clone(totalCapacity));
-        RLESparseResourceAllocation totRLEAvail =
-            new RLESparseResourceAllocation(totAvailable, resCalc);
-
-        // subtract used from available
-        RLESparseResourceAllocation netAvailable;
-
-        netAvailable = RLESparseResourceAllocation.merge(resCalc,
-            Resources.clone(totalCapacity), totRLEAvail, rleSparseVector,
-            RLEOperator.subtractTestNonNegative, start, end);
-
-        // remove periodic component
-        netAvailable = RLESparseResourceAllocation.merge(resCalc,
-            Resources.clone(totalCapacity), netAvailable, periodicRle,
-            RLEOperator.subtractTestNonNegative, start, end);
-
-        // add back in old reservation used resources if any
-        ReservationAllocation old = reservationTable.get(oldId);
-        if (old != null) {
-
-          RLESparseResourceAllocation addBackPrevious =
-              old.getResourcesOverTime(start, end);
-          netAvailable = RLESparseResourceAllocation.merge(resCalc,
-              Resources.clone(totalCapacity), netAvailable, addBackPrevious,
-              RLEOperator.add, start, end);
-        }
-        // lower it if this is needed by the sharing policy
-        netAvailable = getSharingPolicy().availableResources(netAvailable, this,
-            user, oldId, start, end);
-        return netAvailable;
-      } else {
-
-        if (periodicRle.getTimePeriod() % period != 0) {
-          throw new PlanningException("The reservation periodicity (" + period
-              + ") must be" + "an exact divider of the system maxPeriod ("
-              + periodicRle.getTimePeriod() + ")");
-        }
-
-        // find the minimum resources available among all the instances that fit
-        // in the LCM
-        long numInstInLCM = periodicRle.getTimePeriod() / period;
-
-        RLESparseResourceAllocation minOverLCM =
-            getAvailableResourceOverTime(user, oldId, start, end, 0);
-        for (int i = 1; i < numInstInLCM; i++) {
-
-          long rStart = start + i * period;
-          long rEnd = end + i * period;
-
-          // recursive invocation of non-periodic range (to pick raw-info)
-          RLESparseResourceAllocation snapShot =
-              getAvailableResourceOverTime(user, oldId, rStart, rEnd, 0);
-
-          // time-align on start
-          snapShot.shift(-(i * period));
-
-          // pick the minimum amount of resources in each time interval
-          minOverLCM =
-              RLESparseResourceAllocation.merge(resCalc, getTotalCapacity(),
-                  minOverLCM, snapShot, RLEOperator.min, start, end);
-
-        }
-
-        return minOverLCM;
-
+      // create RLE of totCapacity
+      TreeMap<Long, Resource> totAvailable = new TreeMap<Long, Resource>();
+      totAvailable.put(start, Resources.clone(totalCapacity));
+      RLESparseResourceAllocation totRLEAvail =
+          new RLESparseResourceAllocation(totAvailable, resCalc);
+
+      // subtract used from available
+      RLESparseResourceAllocation netAvailable;
+
+      netAvailable =
+          RLESparseResourceAllocation.merge(resCalc,
+              Resources.clone(totalCapacity), totRLEAvail, rleSparseVector,
+              RLEOperator.subtractTestNonNegative, start, end);
+
+      // add back in old reservation used resources if any
+      ReservationAllocation old = reservationTable.get(oldId);
+      if (old != null) {
+        netAvailable =
+            RLESparseResourceAllocation.merge(resCalc,
+                Resources.clone(totalCapacity), netAvailable,
+                old.getResourcesOverTime(), RLEOperator.add, start, end);
       }
+      // lower it if this is needed by the sharing policy
+      netAvailable =
+          getSharingPolicy().availableResources(netAvailable, this, user,
+              oldId, start, end);
+      return netAvailable;
     } finally {
       readLock.unlock();
     }
@@ -808,7 +637,7 @@ public class InMemoryPlan implements Plan {
   public String toCumulativeString() {
     readLock.lock();
     try {
-      return rleSparseVector.toString() + "\n" + periodicRle.toString();
+      return rleSparseVector.toString();
     } finally {
       readLock.unlock();
     }
@@ -860,18 +689,11 @@ public class InMemoryPlan implements Plan {
   }
 
   @Override
-  public RLESparseResourceAllocation getCumulativeLoadOverTime(long start,
-      long end) throws PlanningException {
+  public RLESparseResourceAllocation getCumulativeLoadOverTime(
+      long start, long end) {
     readLock.lock();
     try {
-
-      RLESparseResourceAllocation ret =
-          rleSparseVector.getRangeOverlapping(start, end);
-      ret = RLESparseResourceAllocation.merge(resCalc, totalCapacity, ret,
-          periodicRle.getRangeOverlapping(start, end), RLEOperator.add, start,
-          end);
-
-      return ret;
+      return rleSparseVector.getRangeOverlapping(start, end);
     } finally {
       readLock.unlock();
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e3345e98/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/InMemoryReservationAllocation.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/reservation/InMemoryReservationAllocation.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/InMemoryReservationAllocation.java
index 00c8e44..69fd43f 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/InMemoryReservationAllocation.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/InMemoryReservationAllocation.java
@@ -42,7 +42,6 @@ public class InMemoryReservationAllocation implements ReservationAllocation {
   private final Map<ReservationInterval, Resource> allocationRequests;
   private boolean hasGang = false;
   private long acceptedAt = -1;
-  private long periodicity = 0;
 
   private RLESparseResourceAllocation resourcesOverTime;
 
@@ -68,16 +67,9 @@ public class InMemoryReservationAllocation implements ReservationAllocation {
     this.allocationRequests = allocations;
     this.planName = planName;
     this.hasGang = hasGang;
-    if (contract != null && contract.getRecurrenceExpression() != null) {
-      this.periodicity = Long.parseLong(contract.getRecurrenceExpression());
-    }
-    if (periodicity > 0) {
-      resourcesOverTime =
-          new PeriodicRLESparseResourceAllocation(calculator, periodicity);
-    } else {
-      resourcesOverTime = new RLESparseResourceAllocation(calculator);
-    }
-    for (Map.Entry<ReservationInterval, Resource> r : allocations.entrySet()) {
+    resourcesOverTime = new RLESparseResourceAllocation(calculator);
+    for (Map.Entry<ReservationInterval, Resource> r : allocations
+        .entrySet()) {
       resourcesOverTime.addInterval(r.getKey(), r.getValue());
     }
   }
@@ -141,33 +133,17 @@ public class InMemoryReservationAllocation implements ReservationAllocation {
   }
 
   @Override
-  public RLESparseResourceAllocation getResourcesOverTime() {
+  public RLESparseResourceAllocation getResourcesOverTime(){
     return resourcesOverTime;
   }
 
   @Override
-  public RLESparseResourceAllocation getResourcesOverTime(long start,
-      long end) {
-    return resourcesOverTime.getRangeOverlapping(start, end);
-  }
-
-  @Override
-  public long getPeriodicity() {
-    return periodicity;
-  }
-
-  @Override
-  public void setPeriodicity(long period) {
-    periodicity = period;
-  }
-
-  @Override
   public String toString() {
     StringBuilder sBuf = new StringBuilder();
     sBuf.append(getReservationId()).append(" user:").append(getUser())
         .append(" startTime: ").append(getStartTime()).append(" endTime: ")
-        .append(getEndTime()).append(" Periodiciy: ").append(periodicity)
-        .append(" alloc:\n[").append(resourcesOverTime.toString()).append("] ");
+        .append(getEndTime()).append(" alloc:\n[")
+        .append(resourcesOverTime.toString()).append("] ");
     return sBuf.toString();
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e3345e98/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/NoOverCommitPolicy.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/reservation/NoOverCommitPolicy.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/NoOverCommitPolicy.java
index 49d4702..55f1d00 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/NoOverCommitPolicy.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/NoOverCommitPolicy.java
@@ -40,7 +40,7 @@ public class NoOverCommitPolicy implements SharingPolicy {
 
     RLESparseResourceAllocation available = plan.getAvailableResourceOverTime(
         reservation.getUser(), reservation.getReservationId(),
-        reservation.getStartTime(), reservation.getEndTime(), 0);
+        reservation.getStartTime(), reservation.getEndTime());
 
     // test the reservation does not exceed what is available
     try {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e3345e98/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/PeriodicRLESparseResourceAllocation.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/reservation/PeriodicRLESparseResourceAllocation.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/PeriodicRLESparseResourceAllocation.java
index 7bc44f5..8e3be8b3 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/PeriodicRLESparseResourceAllocation.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/PeriodicRLESparseResourceAllocation.java
@@ -18,94 +18,47 @@
 
 package org.apache.hadoop.yarn.server.resourcemanager.reservation;
 
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Map;
-import java.util.NavigableMap;
-import java.util.TreeMap;
-
 import org.apache.hadoop.yarn.api.records.Resource;
-import org.apache.hadoop.yarn.conf.YarnConfiguration;
-import org.apache.hadoop.yarn.util.resource.ResourceCalculator;
 import org.apache.hadoop.yarn.util.resource.Resources;
+
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import com.google.common.annotations.VisibleForTesting;
-
 /**
- * This data structure stores a periodic {@link RLESparseResourceAllocation}.
+ * This data structure stores a periodic RLESparseResourceAllocation.
  * Default period is 1 day (86400000ms).
  */
-public class PeriodicRLESparseResourceAllocation
-    extends RLESparseResourceAllocation {
+public class PeriodicRLESparseResourceAllocation extends
+    RLESparseResourceAllocation {
 
   // Log
-  private static final Logger LOG =
-      LoggerFactory.getLogger(PeriodicRLESparseResourceAllocation.class);
+  private static final Logger LOG = LoggerFactory
+      .getLogger(PeriodicRLESparseResourceAllocation.class);
 
   private long timePeriod;
 
   /**
    * Constructor.
    *
-   * @param resourceCalculator {@link ResourceCalculator} the resource
-   *          calculator to use.
+   * @param rleVector {@link RLESparseResourceAllocation} with the run-length
+              encoded data.
    * @param timePeriod Time period in milliseconds.
    */
   public PeriodicRLESparseResourceAllocation(
-      ResourceCalculator resourceCalculator, Long timePeriod) {
-    super(resourceCalculator);
+      RLESparseResourceAllocation rleVector, Long timePeriod) {
+    super(rleVector.getCumulative(), rleVector.getResourceCalculator());
     this.timePeriod = timePeriod;
   }
 
   /**
    * Constructor. Default time period set to 1 day.
    *
-   * @param resourceCalculator {@link ResourceCalculator} the resource
-   *          calculator to use..
-   */
-  public PeriodicRLESparseResourceAllocation(
-      ResourceCalculator resourceCalculator) {
-    this(resourceCalculator,
-        YarnConfiguration.DEFAULT_RM_RESERVATION_SYSTEM_MAX_PERIODICITY);
-  }
-
-  /**
-   * Constructor.
-   *
    * @param rleVector {@link RLESparseResourceAllocation} with the run-length
-   *          encoded data.
-   * @param timePeriod Time period in milliseconds.
+              encoded data.
    */
-  @VisibleForTesting
   public PeriodicRLESparseResourceAllocation(
-      RLESparseResourceAllocation rleVector, Long timePeriod) {
-    super(rleVector.getCumulative(), rleVector.getResourceCalculator());
-    this.timePeriod = timePeriod;
-
-    // make sure the PeriodicRLE is zero-based, and handles wrap-around
-    long delta = (getEarliestStartTime() % timePeriod - getEarliestStartTime());
-    shift(delta);
-
-    List<Long> toRemove = new ArrayList<>();
-    Map<Long, Resource> toAdd = new TreeMap<>();
-
-    for (Map.Entry<Long, Resource> entry : cumulativeCapacity.entrySet()) {
-      if (entry.getKey() > timePeriod) {
-        toRemove.add(entry.getKey());
-        if (entry.getValue() != null) {
-          toAdd.put(timePeriod, entry.getValue());
-          long prev = entry.getKey() % timePeriod;
-          toAdd.put(prev, this.getCapacityAtTime(prev));
-          toAdd.put(0L, entry.getValue());
-        }
-      }
-    }
-    for (Long l : toRemove) {
-      cumulativeCapacity.remove(l);
-    }
-    cumulativeCapacity.putAll(toAdd);
+      RLESparseResourceAllocation rleVector) {
+    this(rleVector, 86400000L);
   }
 
   /**
@@ -125,25 +78,24 @@ public class PeriodicRLESparseResourceAllocation
    * The interval may include 0, but the end time must be strictly less than
    * timePeriod.
    *
-   * @param interval {@link ReservationInterval} to which the specified resource
-   *          is to be added.
+   * @param interval {@link ReservationInterval} to which the specified
+   *          resource is to be added.
    * @param resource {@link Resource} to be added to the interval specified.
    * @return true if addition is successful, false otherwise
    */
-  public boolean addInterval(ReservationInterval interval, Resource resource) {
+  public boolean addInterval(ReservationInterval interval,
+      Resource resource) {
     long startTime = interval.getStartTime();
     long endTime = interval.getEndTime();
-
     if (startTime >= 0 && endTime > startTime && endTime <= timePeriod) {
       return super.addInterval(interval, resource);
     } else {
-      LOG.info("Cannot set capacity beyond end time: " + timePeriod + " was ("
-          + interval.toString() + ")");
+      LOG.info("Cannot set capacity beyond end time: " + timePeriod);
       return false;
     }
   }
 
-  /**
+   /**
    * Removes a resource for the specified interval.
    *
    * @param interval the {@link ReservationInterval} for which the resource is
@@ -151,15 +103,14 @@ public class PeriodicRLESparseResourceAllocation
    * @param resource the {@link Resource} to be removed.
    * @return true if removal is successful, false otherwise
    */
-  public boolean removeInterval(ReservationInterval interval,
-      Resource resource) {
+  public boolean removeInterval(
+      ReservationInterval interval, Resource resource) {
     long startTime = interval.getStartTime();
     long endTime = interval.getEndTime();
     // If the resource to be subtracted is less than the minimum resource in
     // the range, abort removal to avoid negative capacity.
-    // TODO revesit decrementing endTime
-    if (!Resources.fitsIn(resource, getMinimumCapacityInInterval(
-        new ReservationInterval(startTime, endTime - 1)))) {
+    if (!Resources.fitsIn(
+        resource, super.getMinimumCapacityInInterval(interval))) {
       LOG.info("Request to remove more resources than what is available");
       return false;
     }
@@ -174,16 +125,17 @@ public class PeriodicRLESparseResourceAllocation
   /**
    * Get maximum capacity at periodic offsets from the specified time.
    *
-   * @param tick UTC time base from which offsets are specified for finding the
-   *          maximum capacity.
-   * @param period periodic offset at which capacities are evaluated.
+   * @param tick UTC time base from which offsets are specified for finding
+   *          the maximum capacity.
+   * @param period periodic offset at which capacities are evaluted.
    * @return the maximum {@link Resource} across the specified time instants.
    * @return true if removal is successful, false otherwise
    */
   public Resource getMaximumPeriodicCapacity(long tick, long period) {
     Resource maxResource;
     if (period < timePeriod) {
-      maxResource = super.getMaximumPeriodicCapacity(tick % timePeriod, period);
+      maxResource =
+          super.getMaximumPeriodicCapacity(tick % timePeriod, period);
     } else {
       // if period is greater than the length of PeriodicRLESparseAllocation,
       // only a single value exists in this interval.
@@ -212,30 +164,4 @@ public class PeriodicRLESparseResourceAllocation
     return ret.toString();
   }
 
-  @Override
-  public RLESparseResourceAllocation getRangeOverlapping(long start, long end) {
-    NavigableMap<Long, Resource> unrolledMap = new TreeMap<>();
-    readLock.lock();
-    try {
-      long relativeStart = (start >= 0) ? start % timePeriod : 0;
-      NavigableMap<Long, Resource> cumulativeMap = this.getCumulative();
-      Long previous = cumulativeMap.floorKey(relativeStart);
-      previous = (previous != null) ? previous : 0;
-      for (long i = 0; i <= (end - start) / timePeriod; i++) {
-        for (Map.Entry<Long, Resource> e : cumulativeMap.entrySet()) {
-          long curKey = e.getKey() + (i * timePeriod);
-          if (curKey >= previous && (start + curKey - relativeStart) <= end) {
-            unrolledMap.put(curKey, e.getValue());
-          }
-        }
-      }
-      RLESparseResourceAllocation rle =
-          new RLESparseResourceAllocation(unrolledMap, getResourceCalculator());
-      rle.shift(start - relativeStart);
-      return rle;
-    } finally {
-      readLock.unlock();
-    }
-  }
-
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e3345e98/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/PlanEdit.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/reservation/PlanEdit.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/PlanEdit.java
index 9afa324..504a250 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/PlanEdit.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/PlanEdit.java
@@ -28,58 +28,54 @@ import org.apache.hadoop.yarn.server.resourcemanager.reservation.exceptions.Plan
 public interface PlanEdit extends PlanContext, PlanView {
 
   /**
-   * Add a new {@link ReservationAllocation} to the plan.
+   * Add a new {@link ReservationAllocation} to the plan
    * 
    * @param reservation the {@link ReservationAllocation} to be added to the
    *          plan
    * @param isRecovering flag to indicate if reservation is being added as part
    *          of failover or not
    * @return true if addition is successful, false otherwise
-   * @throws PlanningException if addition is unsuccessful
    */
-  boolean addReservation(ReservationAllocation reservation,
+  public boolean addReservation(ReservationAllocation reservation,
       boolean isRecovering) throws PlanningException;
 
   /**
    * Updates an existing {@link ReservationAllocation} in the plan. This is
-   * required for re-negotiation.
+   * required for re-negotiation
    * 
    * @param reservation the {@link ReservationAllocation} to be updated the plan
    * @return true if update is successful, false otherwise
-   * @throws PlanningException if update is unsuccessful
    */
-  boolean updateReservation(ReservationAllocation reservation)
+  public boolean updateReservation(ReservationAllocation reservation)
       throws PlanningException;
 
   /**
    * Delete an existing {@link ReservationAllocation} from the plan identified
    * uniquely by its {@link ReservationId}. This will generally be used for
-   * garbage collection.
+   * garbage collection
    * 
    * @param reservationID the {@link ReservationAllocation} to be deleted from
    *          the plan identified uniquely by its {@link ReservationId}
    * @return true if delete is successful, false otherwise
-   * @throws PlanningException if deletion is unsuccessful
    */
-  boolean deleteReservation(ReservationId reservationID)
+  public boolean deleteReservation(ReservationId reservationID)
       throws PlanningException;
 
   /**
    * Method invoked to garbage collect old reservations. It cleans up expired
-   * reservations that have fallen out of the sliding archival window.
+   * reservations that have fallen out of the sliding archival window
    * 
    * @param tick the current time from which the archival window is computed
-   * @throws PlanningException if archival is unsuccessful
    */
-  void archiveCompletedReservations(long tick) throws PlanningException;
+  public void archiveCompletedReservations(long tick) throws PlanningException;
 
   /**
    * Sets the overall capacity in terms of {@link Resource} assigned to this
-   * plan.
+   * plan
    * 
    * @param capacity the overall capacity in terms of {@link Resource} assigned
    *          to this plan
    */
-  void setTotalCapacity(Resource capacity);
+  public void setTotalCapacity(Resource capacity);
 
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e3345e98/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/PlanView.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/reservation/PlanView.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/PlanView.java
index 4035f68..2767993 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/PlanView.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/PlanView.java
@@ -17,50 +17,50 @@
  */
 package org.apache.hadoop.yarn.server.resourcemanager.reservation;
 
-import java.util.Set;
-
 import org.apache.hadoop.yarn.api.records.ReservationId;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.server.resourcemanager.reservation.exceptions.PlanningException;
 
+import java.util.Set;
+
 /**
  * This interface provides a read-only view on the allocations made in this
  * plan. This methods are used for example by {@code ReservationAgent}s to
  * determine the free resources in a certain point in time, and by
  * PlanFollowerPolicy to publish this plan to the scheduler.
  */
-interface PlanView extends PlanContext {
+public interface PlanView extends PlanContext {
 
   /**
    * Return a set of {@link ReservationAllocation} identified by the user who
    * made the reservation.
    *
    * @param reservationID the unqiue id to identify the
-   *          {@link ReservationAllocation}
+   * {@link ReservationAllocation}
    * @param interval the time interval used to retrieve the reservation
-   *          allocations from. Only reservations with start time no greater
-   *          than the interval end time, and end time no less than the interval
-   *          start time will be selected.
+   *                 allocations from. Only reservations with start time no
+   *                 greater than the interval end time, and end time no less
+   *                 than the interval start time will be selected.
    * @param user the user to retrieve the reservation allocation from.
    * @return a set of {@link ReservationAllocation} identified by the user who
-   *         made the reservation
+   * made the reservation
    */
-  Set<ReservationAllocation> getReservations(ReservationId reservationID,
-      ReservationInterval interval, String user);
+  Set<ReservationAllocation> getReservations(ReservationId
+                    reservationID, ReservationInterval interval, String user);
 
   /**
    * Return a set of {@link ReservationAllocation} identified by any user.
    *
    * @param reservationID the unqiue id to identify the
-   *          {@link ReservationAllocation}
+   * {@link ReservationAllocation}
    * @param interval the time interval used to retrieve the reservation
-   *          allocations from. Only reservations with start time no greater
-   *          than the interval end time, and end time no less than the interval
-   *          start time will be selected.
+   *                 allocations from. Only reservations with start time no
+   *                 greater than the interval end time, and end time no less
+   *                 than the interval start time will be selected.
    * @return a set of {@link ReservationAllocation} identified by any user
    */
   Set<ReservationAllocation> getReservations(ReservationId reservationID,
-      ReservationInterval interval);
+                    ReservationInterval interval);
 
   /**
    * Return a {@link ReservationAllocation} identified by its
@@ -70,7 +70,7 @@ interface PlanView extends PlanContext {
    *          {@link ReservationAllocation}
    * @return {@link ReservationAllocation} identified by the specified id
    */
-  ReservationAllocation getReservationById(ReservationId reservationID);
+  public ReservationAllocation getReservationById(ReservationId reservationID);
 
   /**
    * Return a set of {@link ReservationAllocation} that belongs to a certain
@@ -78,10 +78,11 @@ interface PlanView extends PlanContext {
    *
    * @param user the user being considered
    * @param t the instant in time being considered
-   * @return set of active {@link ReservationAllocation}s for this user at this
-   *         time
+   * @return set of active {@link ReservationAllocation}s for this
+   *         user at this time
    */
-  Set<ReservationAllocation> getReservationByUserAtTime(String user, long t);
+  public Set<ReservationAllocation> getReservationByUserAtTime(String user,
+      long t);
 
   /**
    * Gets all the active reservations at the specified point of time
@@ -90,14 +91,14 @@ interface PlanView extends PlanContext {
    *          requested
    * @return set of active reservations at the specified time
    */
-  Set<ReservationAllocation> getReservationsAtTime(long tick);
+  public Set<ReservationAllocation> getReservationsAtTime(long tick);
 
   /**
    * Gets all the reservations in the plan
    * 
    * @return set of all reservations handled by this Plan
    */
-  Set<ReservationAllocation> getAllReservations();
+  public Set<ReservationAllocation> getAllReservations();
 
   /**
    * Returns the total {@link Resource} reserved for all users at the specified
@@ -125,68 +126,61 @@ interface PlanView extends PlanContext {
    * 
    * @return the time (UTC in ms) at which the first reservation starts
    */
-  long getEarliestStartTime();
+  public long getEarliestStartTime();
 
   /**
    * Returns the time (UTC in ms) at which the last reservation terminates
    *
    * @return the time (UTC in ms) at which the last reservation terminates
    */
-  long getLastEndTime();
+  public long getLastEndTime();
 
   /**
    * This method returns the amount of resources available to a given user
    * (optionally if removing a certain reservation) over the start-end time
-   * range. If the request is periodic (period is non-zero) we return the
-   * minimum amount of resources available to periodic reservations (in all
-   * "period" windows within the system maxPeriod / LCM).
+   * range.
    *
-   * @param user the user being considered
-   * @param oldId the identifier of the existing reservation
-   * @param start start of the time interval.
-   * @param end end of the time interval.
-   * @param period the ms periodicty for this request (loop and pick min till
-   *          maxPeriodicity)
+   * @param user
+   * @param oldId
+   * @param start
+   * @param end
    * @return a view of the plan as it is available to this user
-   * @throws PlanningException if operation is unsuccessful
+   * @throws PlanningException
    */
-  RLESparseResourceAllocation getAvailableResourceOverTime(String user,
-      ReservationId oldId, long start, long end, long period)
-      throws PlanningException;
+  public RLESparseResourceAllocation getAvailableResourceOverTime(String user,
+      ReservationId oldId, long start, long end) throws PlanningException;
 
   /**
    * This method returns a RLE encoded view of the user reservation count
    * utilization between start and end time.
    *
-   * @param user the user being considered
-   * @param start start of the time interval.
-   * @param end end of the time interval.
+   * @param user
+   * @param start
+   * @param end
    * @return RLE encoded view of reservation used over time
    */
-  RLESparseResourceAllocation getReservationCountForUserOverTime(String user,
-      long start, long end);
+  public RLESparseResourceAllocation getReservationCountForUserOverTime(
+      String user, long start, long end);
 
   /**
    * This method returns a RLE encoded view of the user reservation utilization
    * between start and end time.
    *
-   * @param user the user being considered
-   * @param start start of the time interval.
-   * @param end end of the time interval.
+   * @param user
+   * @param start
+   * @param end
    * @return RLE encoded view of resources used over time
    */
-  RLESparseResourceAllocation getConsumptionForUserOverTime(String user,
+  public RLESparseResourceAllocation getConsumptionForUserOverTime(String user,
       long start, long end);
 
   /**
    * Get the cumulative load over a time interval.
    *
-   * @param start start of the time interval.
-   * @param end end of the time interval.
+   * @param start Start of the time interval.
+   * @param end End of the time interval.
    * @return RLE sparse allocation.
-   * @throws PlanningException if operation is unsuccessful
    */
-  RLESparseResourceAllocation getCumulativeLoadOverTime(long start, long end)
-      throws PlanningException;
+  RLESparseResourceAllocation getCumulativeLoadOverTime(long start, long end);
 
 }


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[32/37] hadoop git commit: YARN-6978. Add updateContainer API to NMClient. (Kartheek Muthyala via asuresh)

Posted by ae...@apache.org.
YARN-6978. Add updateContainer API to NMClient. (Kartheek Muthyala via asuresh)


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

Branch: refs/heads/HDFS-7240
Commit: c41118a7f826dcbb269b4bd3d5877b35fbbee2b1
Parents: 13eda50
Author: Arun Suresh <as...@apache.org>
Authored: Thu Sep 7 10:23:12 2017 -0700
Committer: Arun Suresh <as...@apache.org>
Committed: Thu Sep 7 10:23:12 2017 -0700

----------------------------------------------------------------------
 .../distributedshell/ApplicationMaster.java     | 19 +++-
 .../apache/hadoop/yarn/client/api/NMClient.java | 18 ++++
 .../yarn/client/api/async/NMClientAsync.java    | 36 ++++++++
 .../api/async/impl/NMClientAsyncImpl.java       | 92 +++++++++++++++-----
 .../yarn/client/api/impl/NMClientImpl.java      | 29 ++++++
 .../api/async/impl/TestNMClientAsync.java       | 60 ++++++++++++-
 .../yarn/client/api/impl/TestAMRMClient.java    |  1 +
 .../yarn/client/api/impl/TestNMClient.java      |  5 +-
 8 files changed, 227 insertions(+), 33 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/c41118a7/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 a02af70..5ec9409 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
@@ -1027,10 +1027,6 @@ public class ApplicationMaster {
     }
 
     @Override
-    public void onContainerResourceIncreased(
-        ContainerId containerId, Resource resource) {}
-
-    @Override
     public void onStartContainerError(ContainerId containerId, Throwable t) {
       LOG.error("Failed to start Container " + containerId, t);
       containers.remove(containerId);
@@ -1050,10 +1046,25 @@ public class ApplicationMaster {
       containers.remove(containerId);
     }
 
+    @Deprecated
     @Override
     public void onIncreaseContainerResourceError(
         ContainerId containerId, Throwable t) {}
 
+    @Deprecated
+    @Override
+    public void onContainerResourceIncreased(
+        ContainerId containerId, Resource resource) {}
+
+    @Override
+    public void onUpdateContainerResourceError(
+        ContainerId containerId, Throwable t) {
+    }
+
+    @Override
+    public void onContainerResourceUpdated(ContainerId containerId,
+        Resource resource) {
+    }
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c41118a7/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/NMClient.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/NMClient.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/NMClient.java
index c1447ba..17168f7 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/NMClient.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/NMClient.java
@@ -104,10 +104,28 @@ public abstract class NMClient extends AbstractService {
    * @throws YarnException YarnException.
    * @throws IOException IOException.
    */
+  @Deprecated
   public abstract void increaseContainerResource(Container container)
       throws YarnException, IOException;
 
   /**
+   * <p>Update the resources of a container.</p>
+   *
+   * <p>The <code>ApplicationMaster</code> or other applications that use the
+   * client must provide the details of the container, including the Id and
+   * the target resource encapsulated in the updated container token via
+   * {@link Container}.
+   * </p>
+   *
+   * @param container the container with updated token.
+   *
+   * @throws YarnException YarnException.
+   * @throws IOException IOException.
+   */
+  public abstract void updateContainerResource(Container container)
+      throws YarnException, IOException;
+
+  /**
    * <p>Stop an started container.</p>
    *
    * @param containerId the Id of the started container

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c41118a7/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/async/NMClientAsync.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/async/NMClientAsync.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/async/NMClientAsync.java
index c94942a..62e2d99 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/async/NMClientAsync.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/async/NMClientAsync.java
@@ -177,9 +177,23 @@ public abstract class NMClientAsync extends AbstractService {
   public abstract void startContainerAsync(
       Container container, ContainerLaunchContext containerLaunchContext);
 
+  @Deprecated
   public abstract void increaseContainerResourceAsync(Container container);
 
   /**
+   * <p>Update the resources of a container.</p>
+   *
+   * <p>The <code>ApplicationMaster</code> or other applications that use the
+   * client must provide the details of the container, including the Id and
+   * the target resource encapsulated in the updated container token via
+   * {@link Container}.
+   * </p>
+   *
+   * @param container the container with updated token.
+   */
+  public abstract void updateContainerResourceAsync(Container container);
+
+  /**
    * <p>Re-Initialize the Container.</p>
    *
    * @param containerId the Id of the container to Re-Initialize.
@@ -301,10 +315,21 @@ public abstract class NMClientAsync extends AbstractService {
      * @param containerId the Id of the container
      * @param resource the target resource of the container
      */
+    @Deprecated
     public abstract void onContainerResourceIncreased(
         ContainerId containerId, Resource resource);
 
     /**
+     * The API is called when <code>NodeManager</code> responds to indicate
+     * the container resource has been successfully updated.
+     *
+     * @param containerId the Id of the container
+     * @param resource the target resource of the container
+     */
+    public abstract void onContainerResourceUpdated(
+        ContainerId containerId, Resource resource);
+
+    /**
      * The API is called when an exception is raised in the process of
      * querying the status of a container.
      *
@@ -321,11 +346,22 @@ public abstract class NMClientAsync extends AbstractService {
      * @param containerId the Id of the container
      * @param t the raised exception
      */
+    @Deprecated
     public abstract void onIncreaseContainerResourceError(
         ContainerId containerId, Throwable t);
 
     /**
      * The API is called when an exception is raised in the process of
+     * updating container resource.
+     *
+     * @param containerId the Id of the container
+     * @param t the raised exception
+     */
+    public abstract void onUpdateContainerResourceError(
+        ContainerId containerId, Throwable t);
+
+    /**
+     * The API is called when an exception is raised in the process of
      * stopping a container.
      *
      * @param containerId the Id of the container

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c41118a7/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/async/impl/NMClientAsyncImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/async/impl/NMClientAsyncImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/async/impl/NMClientAsyncImpl.java
index 20be71e..4d14180 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/async/impl/NMClientAsyncImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/async/impl/NMClientAsyncImpl.java
@@ -259,6 +259,7 @@ public class NMClientAsyncImpl extends NMClientAsync {
     }
   }
 
+  @Deprecated
   public void increaseContainerResourceAsync(Container container) {
     if (!(callbackHandler instanceof AbstractCallbackHandler)) {
       LOG.error("Callback handler does not implement container resource "
@@ -274,7 +275,7 @@ public class NMClientAsyncImpl extends NMClientAsync {
                   " is neither started nor scheduled to start"));
     }
     try {
-      events.put(new IncreaseContainerResourceEvent(container));
+      events.put(new UpdateContainerResourceEvent(container, true));
     } catch (InterruptedException e) {
       LOG.warn("Exception when scheduling the event of increasing resource of "
           + "Container " + container.getId());
@@ -283,6 +284,30 @@ public class NMClientAsyncImpl extends NMClientAsync {
   }
 
   @Override
+  public void updateContainerResourceAsync(Container container) {
+    if (!(callbackHandler instanceof AbstractCallbackHandler)) {
+      LOG.error("Callback handler does not implement container resource "
+          + "increase callback methods");
+      return;
+    }
+    AbstractCallbackHandler handler = (AbstractCallbackHandler) callbackHandler;
+    if (containers.get(container.getId()) == null) {
+      handler.onUpdateContainerResourceError(
+          container.getId(),
+          RPCUtil.getRemoteException(
+              "Container " + container.getId() +
+                  " is neither started nor scheduled to start"));
+    }
+    try {
+      events.put(new UpdateContainerResourceEvent(container, false));
+    } catch (InterruptedException e) {
+      LOG.warn("Exception when scheduling the event of increasing resource of "
+          + "Container " + container.getId());
+      handler.onUpdateContainerResourceError(container.getId(), e);
+    }
+  }
+
+  @Override
   public void reInitializeContainerAsync(ContainerId containerId,
       ContainerLaunchContext containerLaunchContex, boolean autoCommit){
     if (!(callbackHandler instanceof AbstractCallbackHandler)) {
@@ -427,7 +452,7 @@ public class NMClientAsyncImpl extends NMClientAsync {
     START_CONTAINER,
     STOP_CONTAINER,
     QUERY_CONTAINER,
-    INCREASE_CONTAINER_RESOURCE,
+    UPDATE_CONTAINER_RESOURCE,
     REINITIALIZE_CONTAINER,
     RESTART_CONTAINER,
     ROLLBACK_LAST_REINIT,
@@ -503,14 +528,20 @@ public class NMClientAsyncImpl extends NMClientAsync {
     }
   }
 
-  protected static class IncreaseContainerResourceEvent extends ContainerEvent {
+  protected static class UpdateContainerResourceEvent extends ContainerEvent {
     private Container container;
+    private boolean isIncreaseEvent;
 
-    public IncreaseContainerResourceEvent(Container container) {
+    // UpdateContainerResourceEvent constructor takes in a
+    // flag to support callback API's calling through the deprecated
+    // increaseContainerResource
+    public UpdateContainerResourceEvent(Container container,
+        boolean isIncreaseEvent) {
       super(container.getId(), container.getNodeId(),
           container.getContainerToken(),
-              ContainerEventType.INCREASE_CONTAINER_RESOURCE);
+          ContainerEventType.UPDATE_CONTAINER_RESOURCE);
       this.container = container;
+      this.isIncreaseEvent = isIncreaseEvent;
     }
 
     public Container getContainer() {
@@ -536,8 +567,8 @@ public class NMClientAsyncImpl extends NMClientAsync {
 
             // Transitions from RUNNING state
             .addTransition(ContainerState.RUNNING, ContainerState.RUNNING,
-                ContainerEventType.INCREASE_CONTAINER_RESOURCE,
-                new IncreaseContainerResourceTransition())
+                ContainerEventType.UPDATE_CONTAINER_RESOURCE,
+                new UpdateContainerResourceTransition())
 
             // Transitions for Container Upgrade
             .addTransition(ContainerState.RUNNING,
@@ -566,7 +597,7 @@ public class NMClientAsyncImpl extends NMClientAsync {
             .addTransition(ContainerState.DONE, ContainerState.DONE,
                 EnumSet.of(ContainerEventType.START_CONTAINER,
                     ContainerEventType.STOP_CONTAINER,
-                    ContainerEventType.INCREASE_CONTAINER_RESOURCE))
+                    ContainerEventType.UPDATE_CONTAINER_RESOURCE))
 
             // Transition from FAILED state
             .addTransition(ContainerState.FAILED, ContainerState.FAILED,
@@ -576,7 +607,7 @@ public class NMClientAsyncImpl extends NMClientAsync {
                     ContainerEventType.RESTART_CONTAINER,
                     ContainerEventType.COMMIT_LAST_REINT,
                     ContainerEventType.ROLLBACK_LAST_REINIT,
-                    ContainerEventType.INCREASE_CONTAINER_RESOURCE));
+                    ContainerEventType.UPDATE_CONTAINER_RESOURCE));
 
     protected static class StartContainerTransition implements
         MultipleArcTransition<StatefulContainer, ContainerEvent,
@@ -628,46 +659,61 @@ public class NMClientAsyncImpl extends NMClientAsync {
       }
     }
 
-    protected static class IncreaseContainerResourceTransition implements
+    protected static class UpdateContainerResourceTransition implements
         SingleArcTransition<StatefulContainer, ContainerEvent> {
+
+      @SuppressWarnings("deprecation")
       @Override
       public void transition(
           StatefulContainer container, ContainerEvent event) {
+        boolean isIncreaseEvent = false;
         if (!(container.nmClientAsync.getCallbackHandler()
             instanceof AbstractCallbackHandler)) {
           LOG.error("Callback handler does not implement container resource "
-              + "increase callback methods");
+              + "update callback methods");
           return;
         }
         AbstractCallbackHandler handler =
             (AbstractCallbackHandler) container.nmClientAsync
                 .getCallbackHandler();
         try {
-          if (!(event instanceof IncreaseContainerResourceEvent)) {
+          if (!(event instanceof UpdateContainerResourceEvent)) {
             throw new AssertionError("Unexpected event type. Expecting:"
-                + "IncreaseContainerResourceEvent. Got:" + event);
+                + "UpdateContainerResourceEvent. Got:" + event);
           }
-          IncreaseContainerResourceEvent increaseEvent =
-              (IncreaseContainerResourceEvent) event;
-          container.nmClientAsync.getClient().increaseContainerResource(
-              increaseEvent.getContainer());
+          UpdateContainerResourceEvent updateEvent =
+              (UpdateContainerResourceEvent) event;
+          container.nmClientAsync.getClient().updateContainerResource(
+              updateEvent.getContainer());
+          isIncreaseEvent = updateEvent.isIncreaseEvent;
           try {
-            handler.onContainerResourceIncreased(
-                increaseEvent.getContainerId(), increaseEvent.getContainer()
-                    .getResource());
+            //If isIncreaseEvent is set, set the appropriate callbacks
+            //for backward compatibility
+            if (isIncreaseEvent) {
+              handler.onContainerResourceIncreased(updateEvent.getContainerId(),
+                  updateEvent.getContainer().getResource());
+            } else {
+              handler.onContainerResourceUpdated(updateEvent.getContainerId(),
+                  updateEvent.getContainer().getResource());
+            }
           } catch (Throwable thr) {
             // Don't process user created unchecked exception
             LOG.info("Unchecked exception is thrown from "
-                + "onContainerResourceIncreased for Container "
+                + "onContainerResourceUpdated for Container "
                 + event.getContainerId(), thr);
           }
         } catch (Exception e) {
           try {
-            handler.onIncreaseContainerResourceError(event.getContainerId(), e);
+            if (isIncreaseEvent) {
+              handler
+                  .onIncreaseContainerResourceError(event.getContainerId(), e);
+            } else {
+              handler.onUpdateContainerResourceError(event.getContainerId(), e);
+            }
           } catch (Throwable thr) {
             // Don't process user created unchecked exception
             LOG.info("Unchecked exception is thrown from "
-                + "onIncreaseContainerResourceError for Container "
+                + "onUpdateContainerResourceError for Container "
                 + event.getContainerId(), thr);
           }
         }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c41118a7/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/impl/NMClientImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/impl/NMClientImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/impl/NMClientImpl.java
index 8171de2..ca44e2e 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/impl/NMClientImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/impl/NMClientImpl.java
@@ -230,6 +230,7 @@ public class NMClientImpl extends NMClient {
     }
   }
 
+  @Deprecated
   @Override
   public void increaseContainerResource(Container container)
       throws YarnException, IOException {
@@ -259,6 +260,34 @@ public class NMClientImpl extends NMClient {
   }
 
   @Override
+  public void updateContainerResource(Container container)
+      throws YarnException, IOException {
+    ContainerManagementProtocolProxyData proxy = null;
+    try {
+      proxy =
+          cmProxy.getProxy(container.getNodeId().toString(), container.getId());
+      List<Token> updateTokens = new ArrayList<>();
+      updateTokens.add(container.getContainerToken());
+
+      ContainerUpdateRequest request =
+          ContainerUpdateRequest.newInstance(updateTokens);
+      ContainerUpdateResponse response =
+          proxy.getContainerManagementProtocol().updateContainer(request);
+
+      if (response.getFailedRequests() != null && response.getFailedRequests()
+          .containsKey(container.getId())) {
+        Throwable t =
+            response.getFailedRequests().get(container.getId()).deSerialize();
+        parseAndThrowException(t);
+      }
+    } finally {
+      if (proxy != null) {
+        cmProxy.mayBeCloseProxy(proxy);
+      }
+    }
+  }
+
+  @Override
   public void stopContainer(ContainerId containerId, NodeId nodeId)
       throws YarnException, IOException {
     StartedContainer startedContainer = startedContainers.get(containerId);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c41118a7/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/async/impl/TestNMClientAsync.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/async/impl/TestNMClientAsync.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/async/impl/TestNMClientAsync.java
index dda3eec..6c72704 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/async/impl/TestNMClientAsync.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/async/impl/TestNMClientAsync.java
@@ -253,7 +253,7 @@ public class TestNMClientAsync {
         int t = containerId.getId() % 5;
         switch (t) {
         case 0:
-          asyncClient.increaseContainerResourceAsync(container);
+          asyncClient.updateContainerResourceAsync(container);
           break;
         case 1:
           asyncClient.reInitializeContainerAsync(containerId,
@@ -295,7 +295,7 @@ public class TestNMClientAsync {
       // containerId
       Container container = Container.newInstance(
           containerId, nodeId, null, null, null, containerToken);
-      asyncClient.increaseContainerResourceAsync(container);
+      asyncClient.updateContainerResourceAsync(container);
 
       // Shouldn't crash the test thread
       throw new RuntimeException("Ignorable Exception");
@@ -322,6 +322,25 @@ public class TestNMClientAsync {
 
     @SuppressWarnings("deprecation")
     @Override
+    public void onContainerResourceUpdated(ContainerId containerId,
+        Resource resource) {
+      if (containerId.getId() >= expectedSuccess) {
+        errorMsgs.add("Container " + containerId +
+            " should throw the exception onContainerResourceUpdated");
+        return;
+      }
+      TestData td = testMap.get(OpsToTest.INCR);
+      td.success.addAndGet(1);
+      td.successArray.set(containerId.getId(), 1);
+      // move on to the following success tests
+      asyncClient.reInitializeContainerAsync(containerId,
+          Records.newRecord(ContainerLaunchContext.class), true);
+      // throw a fake user exception, and shouldn't crash the test
+      throw new RuntimeException("Ignorable Exception");
+    }
+
+    @SuppressWarnings("deprecation")
+    @Override
     public void onContainerReInitialize(ContainerId containerId) {
       if (containerId.getId() >= expectedSuccess) {
         errorMsgs.add("Container " + containerId +
@@ -452,6 +471,27 @@ public class TestNMClientAsync {
 
     @SuppressWarnings("deprecation")
     @Override
+    public void onUpdateContainerResourceError(ContainerId containerId,
+        Throwable t) {
+      if (containerId.getId() < expectedSuccess + expectedFailure) {
+        errorMsgs.add("Container " + containerId +
+            " shouldn't throw the exception onUpdatedContainerResourceError");
+        return;
+      }
+      TestData td = testMap.get(OpsToTest.INCR);
+      td.failure.addAndGet(1);
+      td.failureArray.set(
+          containerId.getId() - expectedSuccess - expectedFailure, 1);
+      // increase container resource error should NOT change the
+      // the container status to FAILED
+      // move on to the following failure tests
+      asyncClient.stopContainerAsync(containerId, nodeId);
+      // Shouldn't crash the test thread
+      throw new RuntimeException("Ignorable Exception");
+    }
+
+    @SuppressWarnings("deprecation")
+    @Override
     public void onContainerReInitializeError(ContainerId containerId,
         Throwable t) {
       if (containerId.getId() < expectedSuccess + expectedFailure) {
@@ -673,7 +713,7 @@ public class TestNMClientAsync {
         when(client.getContainerStatus(any(ContainerId.class),
             any(NodeId.class))).thenReturn(
                 recordFactory.newRecordInstance(ContainerStatus.class));
-        doNothing().when(client).increaseContainerResource(
+        doNothing().when(client).updateContainerResource(
             any(Container.class));
         doNothing().when(client).reInitializeContainer(
             any(ContainerId.class), any(ContainerLaunchContext.class),
@@ -703,7 +743,7 @@ public class TestNMClientAsync {
             any(NodeId.class))).thenReturn(
                 recordFactory.newRecordInstance(ContainerStatus.class));
         doThrow(RPCUtil.getRemoteException("Increase Resource Exception"))
-            .when(client).increaseContainerResource(any(Container.class));
+            .when(client).updateContainerResource(any(Container.class));
         doThrow(RPCUtil.getRemoteException("ReInitialize Exception"))
             .when(client).reInitializeContainer(
             any(ContainerId.class), any(ContainerLaunchContext.class),
@@ -818,11 +858,17 @@ public class TestNMClientAsync {
         ContainerStatus containerStatus) {
     }
 
+    @Deprecated
     @Override
     public void onContainerResourceIncreased(
         ContainerId containerId, Resource resource) {}
 
     @Override
+    public void onContainerResourceUpdated(ContainerId containerId,
+        Resource resource) {
+    }
+
+    @Override
     public void onContainerStopped(ContainerId containerId) {
     }
 
@@ -847,11 +893,17 @@ public class TestNMClientAsync {
         Throwable t) {
     }
 
+    @Deprecated
     @Override
     public void onIncreaseContainerResourceError(
         ContainerId containerId, Throwable t) {}
 
     @Override
+    public void onUpdateContainerResourceError(ContainerId containerId,
+        Throwable t) {
+    }
+
+    @Override
     public void onStopContainerError(ContainerId containerId, Throwable t) {
     }
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c41118a7/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestAMRMClient.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestAMRMClient.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestAMRMClient.java
index 09b12f2..fa2e7a5 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestAMRMClient.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestAMRMClient.java
@@ -1428,6 +1428,7 @@ public class TestAMRMClient {
     amClient.ask.clear();
   }
 
+  @SuppressWarnings("deprecation")
   private void updateContainerExecType(AllocateResponse allocResponse,
       ExecutionType expectedExecType, NMClientImpl nmClient)
       throws IOException, YarnException {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c41118a7/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestNMClient.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestNMClient.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestNMClient.java
index 9b79e2d..b23a923 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestNMClient.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestNMClient.java
@@ -301,10 +301,10 @@ public class TestNMClient {
         assertTrue("The thrown exception is not expected",
             e.getMessage().contains("is not handled by this NodeManager"));
       }
-      // increaseContainerResource shouldn't be called before startContainer,
+      // upadateContainerResource shouldn't be called before startContainer,
       // otherwise, NodeManager cannot find the container
       try {
-        nmClient.increaseContainerResource(container);
+        nmClient.updateContainerResource(container);
         fail("Exception is expected");
       } catch (YarnException e) {
         assertTrue("The thrown exception is not expected",
@@ -469,6 +469,7 @@ public class TestNMClient {
     }
   }
 
+  @SuppressWarnings("deprecation")
   private void testIncreaseContainerResource(Container container)
     throws YarnException, IOException {
     try {


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[19/37] hadoop git commit: YARN-7164. TestAMRMClientOnRMRestart fails sporadically with bind address in use. Contributed by Jason Lowe

Posted by ae...@apache.org.
YARN-7164. TestAMRMClientOnRMRestart fails sporadically with bind address in use. Contributed by Jason Lowe


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

Branch: refs/heads/HDFS-7240
Commit: 704267cb4909bf9968dd30067617f74543e07ffb
Parents: 63231a2
Author: Jason Lowe <jl...@apache.org>
Authored: Wed Sep 6 15:23:49 2017 -0500
Committer: Jason Lowe <jl...@apache.org>
Committed: Wed Sep 6 15:23:49 2017 -0500

----------------------------------------------------------------------
 .../hadoop/yarn/client/api/impl/TestAMRMClientOnRMRestart.java   | 4 +---
 1 file changed, 1 insertion(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/704267cb/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestAMRMClientOnRMRestart.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestAMRMClientOnRMRestart.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestAMRMClientOnRMRestart.java
index fa3c6af..337d7d4 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestAMRMClientOnRMRestart.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestAMRMClientOnRMRestart.java
@@ -29,7 +29,6 @@ import java.util.List;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.net.NetworkTopology;
-import org.apache.hadoop.net.ServerSocketUtil;
 import org.apache.hadoop.security.SecurityUtil;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.token.SecretManager.InvalidToken;
@@ -93,6 +92,7 @@ public class TestAMRMClientOnRMRestart {
         YarnConfiguration.DEFAULT_RM_AM_MAX_ATTEMPTS);
     conf.setBoolean(YarnConfiguration.RM_WORK_PRESERVING_RECOVERY_ENABLED, true);
     conf.setLong(YarnConfiguration.RM_WORK_PRESERVING_RECOVERY_SCHEDULING_WAIT_MS, 0);
+    conf.set(YarnConfiguration.RM_SCHEDULER_ADDRESS, "0.0.0.0:0");
   }
 
   @AfterClass
@@ -496,8 +496,6 @@ public class TestAMRMClientOnRMRestart {
       .getMasterKey().getKeyId() == newToken.decodeIdentifier().getKeyId());
 
     // start 2nd RM
-    conf.set(YarnConfiguration.RM_SCHEDULER_ADDRESS, "0.0.0.0:"
-        + ServerSocketUtil.getPort(45020, 10));
     final MyResourceManager2 rm2 = new MyResourceManager2(conf, memStore);
     rm2.start();
     nm1.setResourceTrackerService(rm2.getResourceTrackerService());


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[37/37] 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/e319be93
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/e319be93
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/e319be93

Branch: refs/heads/HDFS-7240
Commit: e319be93b9d1aeef76efd6c8e9cc2b8efc3502dc
Parents: 5cd6d7b a4cd101
Author: Anu Engineer <ae...@apache.org>
Authored: Thu Sep 7 14:35:56 2017 -0700
Committer: Anu Engineer <ae...@apache.org>
Committed: Thu Sep 7 14:35:56 2017 -0700

----------------------------------------------------------------------
 .../org/apache/hadoop/conf/Configuration.java   |  116 +-
 .../org/apache/hadoop/conf/CorePropertyTag.java |   37 +
 .../org/apache/hadoop/conf/HDFSPropertyTag.java |   41 +
 .../org/apache/hadoop/conf/PropertyTag.java     |   30 +
 .../org/apache/hadoop/conf/YarnPropertyTag.java |   39 +
 .../java/org/apache/hadoop/util/StopWatch.java  |   17 +-
 .../apache/hadoop/conf/TestConfiguration.java   |  112 +-
 .../hdfs/protocol/AddECPolicyResponse.java      |    4 +-
 .../hdfs/protocol/IllegalECPolicyException.java |   34 -
 .../org/apache/hadoop/hdfs/DFSConfigKeys.java   |    5 +-
 .../namenode/ErasureCodingPolicyManager.java    |   39 +-
 .../server/namenode/FSDirErasureCodingOp.java   |    3 +-
 .../hdfs/server/namenode/FSDirectory.java       |   58 +-
 .../hdfs/server/namenode/FSNamesystem.java      |    3 +-
 .../src/main/resources/hdfs-default.xml         |   14 +-
 .../hadoop/hdfs/TestDistributedFileSystem.java  |    4 +-
 .../namenode/TestINodeAttributeProvider.java    |  105 +-
 .../apache/hadoop/fs/s3a/S3AInputStream.java    |    2 +-
 .../fs/azure/AzureNativeFileSystemStore.java    |   73 +-
 .../hadoop/fs/azure/BlockBlobAppendStream.java  | 1301 +++++++++++-------
 .../hadoop/fs/azure/NativeAzureFileSystem.java  |   77 +-
 .../hadoop/fs/azure/NativeFileSystemStore.java  |    5 +-
 .../fs/azure/SecureStorageInterfaceImpl.java    |   10 +-
 .../hadoop/fs/azure/SelfRenewingLease.java      |   10 +-
 .../hadoop/fs/azure/StorageInterface.java       |    3 +-
 .../hadoop/fs/azure/StorageInterfaceImpl.java   |   12 +-
 .../fs/azure/SyncableDataOutputStream.java      |   11 +
 .../hadoop-azure/src/site/markdown/index.md     |   34 +
 .../hadoop/fs/azure/MockStorageInterface.java   |    3 +-
 .../azure/TestAzureConcurrentOutOfBandIo.java   |    6 +-
 ...estNativeAzureFileSystemBlockCompaction.java |  266 ++++
 .../src/test/resources/log4j.properties         |    1 +
 .../distributedshell/ApplicationMaster.java     |   19 +-
 .../apache/hadoop/yarn/client/api/NMClient.java |   18 +
 .../yarn/client/api/async/NMClientAsync.java    |   36 +
 .../api/async/impl/NMClientAsyncImpl.java       |   92 +-
 .../yarn/client/api/impl/NMClientImpl.java      |   29 +
 .../api/async/impl/TestNMClientAsync.java       |   60 +-
 .../yarn/client/api/impl/TestAMRMClient.java    |    1 +
 .../api/impl/TestAMRMClientOnRMRestart.java     |    4 +-
 .../yarn/client/api/impl/TestNMClient.java      |    5 +-
 .../hadoop/yarn/client/cli/TestLogsCLI.java     |   16 +-
 .../logaggregation/AggregatedLogFormat.java     |  105 +-
 .../logaggregation/LogAggregationUtils.java     |   32 +-
 .../yarn/logaggregation/LogCLIHelpers.java      |   41 +-
 .../LogAggregationFileController.java           |   21 +-
 .../tfile/LogAggregationTFileController.java    |    1 -
 .../hadoop/yarn/server/webapp/AppBlock.java     |    9 +-
 .../RLESparseResourceAllocation.java            |    5 +-
 49 files changed, 2215 insertions(+), 754 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/e319be93/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestAMRMClient.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e319be93/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/RLESparseResourceAllocation.java
----------------------------------------------------------------------
diff --cc hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/RLESparseResourceAllocation.java
index 100d38c,3062f3d..57e8104
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/RLESparseResourceAllocation.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/RLESparseResourceAllocation.java
@@@ -409,7 -408,7 +408,7 @@@ public class RLESparseResourceAllocatio
        // test for negative value and throws
        if (operator == RLEOperator.subtractTestNonNegative
            && (Resources.fitsIn(val, ZERO_RESOURCE)
--              && !Resources.equals(val, ZERO_RESOURCE))) {
++          && !Resources.equals(val, ZERO_RESOURCE))) {
          throw new PlanningException(
              "RLESparseResourceAllocation: merge failed as the "
                  + "resulting RLESparseResourceAllocation would be negative");


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[26/37] hadoop git commit: YARN-5328. Plan/ResourceAllocation data structure enhancements required to support recurring reservations in ReservationSystem.

Posted by ae...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/b6e7d136/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/RLESparseResourceAllocation.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/reservation/RLESparseResourceAllocation.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/RLESparseResourceAllocation.java
index 658387b..3062f3d 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/RLESparseResourceAllocation.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/RLESparseResourceAllocation.java
@@ -18,8 +18,6 @@
 
 package org.apache.hadoop.yarn.server.resourcemanager.reservation;
 
-import java.io.IOException;
-import java.io.StringWriter;
 import java.util.Iterator;
 import java.util.Map;
 import java.util.Map.Entry;
@@ -33,8 +31,6 @@ import org.apache.hadoop.yarn.server.resourcemanager.reservation.exceptions.Plan
 import org.apache.hadoop.yarn.util.resource.ResourceCalculator;
 import org.apache.hadoop.yarn.util.resource.Resources;
 
-import com.google.gson.stream.JsonWriter;
-
 /**
  * This is a run length encoded sparse data structure that maintains resource
  * allocations over time.
@@ -44,12 +40,14 @@ public class RLESparseResourceAllocation {
   private static final int THRESHOLD = 100;
   private static final Resource ZERO_RESOURCE = Resources.none();
 
-  private NavigableMap<Long, Resource> cumulativeCapacity =
+  @SuppressWarnings("checkstyle:visibilitymodifier")
+  protected NavigableMap<Long, Resource> cumulativeCapacity =
       new TreeMap<Long, Resource>();
 
   private final ReentrantReadWriteLock readWriteLock =
       new ReentrantReadWriteLock();
-  private final Lock readLock = readWriteLock.readLock();
+  @SuppressWarnings("checkstyle:visibilitymodifier")
+  protected final Lock readLock = readWriteLock.readLock();
   private final Lock writeLock = readWriteLock.writeLock();
 
   private final ResourceCalculator resourceCalculator;
@@ -236,34 +234,6 @@ public class RLESparseResourceAllocation {
   }
 
   /**
-   * Returns the JSON string representation of the current resources allocated
-   * over time.
-   *
-   * @return the JSON string representation of the current resources allocated
-   *         over time
-   */
-  public String toMemJSONString() {
-    StringWriter json = new StringWriter();
-    JsonWriter jsonWriter = new JsonWriter(json);
-    readLock.lock();
-    try {
-      jsonWriter.beginObject();
-      // jsonWriter.name("timestamp").value("resource");
-      for (Map.Entry<Long, Resource> r : cumulativeCapacity.entrySet()) {
-        jsonWriter.name(r.getKey().toString()).value(r.getValue().toString());
-      }
-      jsonWriter.endObject();
-      jsonWriter.close();
-      return json.toString();
-    } catch (IOException e) {
-      // This should not happen
-      return "";
-    } finally {
-      readLock.unlock();
-    }
-  }
-
-  /**
    * Returns the representation of the current resources allocated over time as
    * an interval map (in the defined non-null range).
    *
@@ -437,8 +407,8 @@ public class RLESparseResourceAllocation {
       Resource val = Resources.negate(e.getValue());
       // test for negative value and throws
       if (operator == RLEOperator.subtractTestNonNegative
-          && (Resources.fitsIn(val, ZERO_RESOURCE) &&
-              !Resources.equals(val, ZERO_RESOURCE))) {
+          && (Resources.fitsIn(val, ZERO_RESOURCE)
+              && !Resources.equals(val, ZERO_RESOURCE))) {
         throw new PlanningException(
             "RLESparseResourceAllocation: merge failed as the "
                 + "resulting RLESparseResourceAllocation would be negative");
@@ -504,22 +474,29 @@ public class RLESparseResourceAllocation {
 
   }
 
+  /**
+   * Get a {@link RLESparseResourceAllocation} view of the {@link Resource}
+   * allocations between the specified start and end times.
+   *
+   * @param start the time from which the {@link Resource} allocations are
+   *          required
+   * @param end the time upto which the {@link Resource} allocations are
+   *          required
+   * @return the overlapping allocations
+   */
   public RLESparseResourceAllocation getRangeOverlapping(long start, long end) {
     readLock.lock();
     try {
       NavigableMap<Long, Resource> a = this.getCumulative();
-
       if (a != null && !a.isEmpty()) {
         // include the portion of previous entry that overlaps start
         if (start > a.firstKey()) {
           long previous = a.floorKey(start);
           a = a.tailMap(previous, true);
         }
-
         if (end < a.lastKey()) {
           a = a.headMap(end, true);
         }
-
       }
       RLESparseResourceAllocation ret =
           new RLESparseResourceAllocation(a, resourceCalculator);
@@ -527,7 +504,33 @@ public class RLESparseResourceAllocation {
     } finally {
       readLock.unlock();
     }
+  }
 
+  /**
+   * This method shifts all the timestamp of the {@link Resource} entries by the
+   * specified "delta".
+   *
+   * @param delta the time by which to shift the {@link Resource} allocations
+   */
+  public void shift(long delta) {
+    writeLock.lock();
+    try {
+      TreeMap<Long, Resource> newCum = new TreeMap<>();
+      long start;
+      for (Map.Entry<Long, Resource> entry : cumulativeCapacity.entrySet()) {
+        if (delta > 0) {
+          start = (entry.getKey() == Long.MAX_VALUE) ? Long.MAX_VALUE
+              : entry.getKey() + delta;
+        } else {
+          start = (entry.getKey() == Long.MIN_VALUE) ? Long.MIN_VALUE
+              : entry.getKey() + delta;
+        }
+        newCum.put(start, entry.getValue());
+      }
+      cumulativeCapacity = newCum;
+    } finally {
+      writeLock.unlock();
+    }
   }
 
   /**
@@ -541,8 +544,8 @@ public class RLESparseResourceAllocation {
   /**
    * Get the maximum capacity across specified time instances. The search-space
    * is specified using the starting value, tick, and the periodic interval for
-   * search. Maximum resource allocation across tick, tick + period,
-   * tick + 2 * period,..., tick + n * period .. is returned.
+   * search. Maximum resource allocation across tick, tick + period, tick + 2 *
+   * period,..., tick + n * period .. is returned.
    *
    * @param tick the starting time instance
    * @param period interval at which capacity is evaluated
@@ -550,14 +553,19 @@ public class RLESparseResourceAllocation {
    */
   public Resource getMaximumPeriodicCapacity(long tick, long period) {
     Resource maxCapacity = ZERO_RESOURCE;
-    if (!cumulativeCapacity.isEmpty()) {
-      Long lastKey = cumulativeCapacity.lastKey();
-      for (long t = tick; t <= lastKey; t = t + period) {
-        maxCapacity = Resources.componentwiseMax(maxCapacity,
-            cumulativeCapacity.floorEntry(t).getValue());
+    readLock.lock();
+    try {
+      if (!cumulativeCapacity.isEmpty()) {
+        Long lastKey = cumulativeCapacity.lastKey();
+        for (long t = tick; t <= lastKey; t = t + period) {
+          maxCapacity = Resources.componentwiseMax(maxCapacity,
+              cumulativeCapacity.floorEntry(t).getValue());
+        }
       }
+      return maxCapacity;
+    } finally {
+      readLock.unlock();
     }
-    return maxCapacity;
   }
 
   /**
@@ -567,17 +575,17 @@ public class RLESparseResourceAllocation {
    * @return minimum resource allocation
    */
   public Resource getMinimumCapacityInInterval(ReservationInterval interval) {
-    Resource minCapacity = Resource.newInstance(
-        Integer.MAX_VALUE, Integer.MAX_VALUE);
+    Resource minCapacity =
+        Resource.newInstance(Integer.MAX_VALUE, Integer.MAX_VALUE);
     long start = interval.getStartTime();
     long end = interval.getEndTime();
     NavigableMap<Long, Resource> capacityRange =
-        this.getRangeOverlapping(start, end).getCumulative();
+        getRangeOverlapping(start, end).getCumulative();
     if (!capacityRange.isEmpty()) {
       for (Map.Entry<Long, Resource> entry : capacityRange.entrySet()) {
         if (entry.getValue() != null) {
-          minCapacity = Resources.componentwiseMin(minCapacity,
-              entry.getValue());
+          minCapacity =
+              Resources.componentwiseMin(minCapacity, entry.getValue());
         }
       }
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b6e7d136/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/ReservationAllocation.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/reservation/ReservationAllocation.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/ReservationAllocation.java
index 0da95ac..bb4a7fb 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/ReservationAllocation.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/ReservationAllocation.java
@@ -24,14 +24,16 @@ import org.apache.hadoop.yarn.api.records.ReservationDefinition;
 import org.apache.hadoop.yarn.api.records.ReservationId;
 import org.apache.hadoop.yarn.api.records.Resource;
 
+import com.google.common.annotations.VisibleForTesting;
+
 /**
  * A ReservationAllocation represents a concrete allocation of resources over
  * time that satisfy a certain {@link ReservationDefinition}. This is used
  * internally by a {@link Plan} to store information about how each of the
  * accepted {@link ReservationDefinition} have been allocated.
  */
-public interface ReservationAllocation extends
-    Comparable<ReservationAllocation> {
+public interface ReservationAllocation
+    extends Comparable<ReservationAllocation> {
 
   /**
    * Returns the unique identifier {@link ReservationId} that represents the
@@ -40,28 +42,28 @@ public interface ReservationAllocation extends
    * @return reservationId the unique identifier {@link ReservationId} that
    *         represents the reservation
    */
-  public ReservationId getReservationId();
+  ReservationId getReservationId();
 
   /**
    * Returns the original {@link ReservationDefinition} submitted by the client
    * 
    * @return the {@link ReservationDefinition} submitted by the client
    */
-  public ReservationDefinition getReservationDefinition();
+  ReservationDefinition getReservationDefinition();
 
   /**
    * Returns the time at which the reservation is activated.
    * 
    * @return the time at which the reservation is activated
    */
-  public long getStartTime();
+  long getStartTime();
 
   /**
    * Returns the time at which the reservation terminates.
    * 
    * @return the time at which the reservation terminates
    */
-  public long getEndTime();
+  long getEndTime();
 
   /**
    * Returns the map of resources requested against the time interval for which
@@ -70,28 +72,28 @@ public interface ReservationAllocation extends
    * @return the allocationRequests the map of resources requested against the
    *         time interval for which they were
    */
-  public Map<ReservationInterval, Resource> getAllocationRequests();
+  Map<ReservationInterval, Resource> getAllocationRequests();
 
   /**
    * Return a string identifying the plan to which the reservation belongs
    * 
    * @return the plan to which the reservation belongs
    */
-  public String getPlanName();
+  String getPlanName();
 
   /**
    * Returns the user who requested the reservation
    * 
    * @return the user who requested the reservation
    */
-  public String getUser();
+  String getUser();
 
   /**
    * Returns whether the reservation has gang semantics or not
    * 
    * @return true if there is a gang request, false otherwise
    */
-  public boolean containsGangs();
+  boolean containsGangs();
 
   /**
    * Sets the time at which the reservation was accepted by the system
@@ -99,14 +101,14 @@ public interface ReservationAllocation extends
    * @param acceptedAt the time at which the reservation was accepted by the
    *          system
    */
-  public void setAcceptanceTimestamp(long acceptedAt);
+  void setAcceptanceTimestamp(long acceptedAt);
 
   /**
    * Returns the time at which the reservation was accepted by the system
    * 
    * @return the time at which the reservation was accepted by the system
    */
-  public long getAcceptanceTime();
+  long getAcceptanceTime();
 
   /**
    * Returns the capacity represented by cumulative resources reserved by the
@@ -116,12 +118,42 @@ public interface ReservationAllocation extends
    *          requested
    * @return the resources reserved at the specified time
    */
-  public Resource getResourcesAtTime(long tick);
+  Resource getResourcesAtTime(long tick);
+
+  /**
+   * Return a RLE representation of used resources.
+   *
+   * @return a RLE encoding of resources allocated over time.
+   */
+  RLESparseResourceAllocation getResourcesOverTime();
+
 
   /**
    * Return a RLE representation of used resources.
+   *
+   * @param start start of the time interval.
+   * @param end end of the time interval.
    * @return a RLE encoding of resources allocated over time.
    */
-  public RLESparseResourceAllocation getResourcesOverTime();
+  RLESparseResourceAllocation getResourcesOverTime(long start, long end);
+
+  /**
+   * Get the periodicity of this reservation representing the time period of the
+   * periodic job. Period is represented in milliseconds for periodic jobs.
+   * Period is 0 for non-periodic jobs.
+   *
+   * @return periodicity of this reservation
+   */
+  long getPeriodicity();
+
+  /**
+   * Set the periodicity of this reservation representing the time period of the
+   * periodic job. Period is represented in milliseconds for periodic jobs.
+   * Period is 0 for non-periodic jobs.
+   *
+   * @param period periodicity of this reservation
+   */
+  @VisibleForTesting
+  void setPeriodicity(long period);
 
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b6e7d136/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/ReservationInputValidator.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/reservation/ReservationInputValidator.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/ReservationInputValidator.java
index 027d066..a66d222 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/ReservationInputValidator.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/ReservationInputValidator.java
@@ -44,6 +44,8 @@ public class ReservationInputValidator {
 
   /**
    * Utility class to validate reservation requests.
+   *
+   * @param clock the {@link Clock} to use
    */
   public ReservationInputValidator(Clock clock) {
     this.clock = clock;
@@ -53,22 +55,21 @@ public class ReservationInputValidator {
       ReservationId reservationId, String auditConstant) throws YarnException {
     // check if the reservation id is valid
     if (reservationId == null) {
-      String message =
-          "Missing reservation id."
-              + " Please try again by specifying a reservation id.";
+      String message = "Missing reservation id."
+          + " Please try again by specifying a reservation id.";
       RMAuditLogger.logFailure("UNKNOWN", auditConstant,
           "validate reservation input", "ClientRMService", message);
       throw RPCUtil.getRemoteException(message);
     }
     String queue = reservationSystem.getQueueForReservation(reservationId);
     String nullQueueErrorMessage =
-            "The specified reservation with ID: " + reservationId
-                    + " is unknown. Please try again with a valid reservation.";
+        "The specified reservation with ID: " + reservationId
+            + " is unknown. Please try again with a valid reservation.";
     String nullPlanErrorMessage = "The specified reservation: " + reservationId
-                            + " is not associated with any valid plan."
-                            + " Please try again with a valid reservation.";
+        + " is not associated with any valid plan."
+        + " Please try again with a valid reservation.";
     return getPlanFromQueue(reservationSystem, queue, auditConstant,
-            nullQueueErrorMessage, nullPlanErrorMessage);
+        nullQueueErrorMessage, nullPlanErrorMessage);
   }
 
   private void validateReservationDefinition(ReservationId reservationId,
@@ -77,17 +78,15 @@ public class ReservationInputValidator {
     String message = "";
     // check if deadline is in the past
     if (contract == null) {
-      message =
-          "Missing reservation definition."
-              + " Please try again by specifying a reservation definition.";
+      message = "Missing reservation definition."
+          + " Please try again by specifying a reservation definition.";
       RMAuditLogger.logFailure("UNKNOWN", auditConstant,
           "validate reservation input definition", "ClientRMService", message);
       throw RPCUtil.getRemoteException(message);
     }
     if (contract.getDeadline() <= clock.getTime()) {
-      message =
-          "The specified deadline: " + contract.getDeadline()
-              + " is the past. Please try again with deadline in the future.";
+      message = "The specified deadline: " + contract.getDeadline()
+          + " is the past. Please try again with deadline in the future.";
       RMAuditLogger.logFailure("UNKNOWN", auditConstant,
           "validate reservation input definition", "ClientRMService", message);
       throw RPCUtil.getRemoteException(message);
@@ -95,18 +94,16 @@ public class ReservationInputValidator {
     // Check if at least one RR has been specified
     ReservationRequests resReqs = contract.getReservationRequests();
     if (resReqs == null) {
-      message =
-          "No resources have been specified to reserve."
-              + "Please try again by specifying the resources to reserve.";
+      message = "No resources have been specified to reserve."
+          + "Please try again by specifying the resources to reserve.";
       RMAuditLogger.logFailure("UNKNOWN", auditConstant,
           "validate reservation input definition", "ClientRMService", message);
       throw RPCUtil.getRemoteException(message);
     }
     List<ReservationRequest> resReq = resReqs.getReservationResources();
     if (resReq == null || resReq.isEmpty()) {
-      message =
-          "No resources have been specified to reserve."
-              + " Please try again by specifying the resources to reserve.";
+      message = "No resources have been specified to reserve."
+          + " Please try again by specifying the resources to reserve.";
       RMAuditLogger.logFailure("UNKNOWN", auditConstant,
           "validate reservation input definition", "ClientRMService", message);
       throw RPCUtil.getRemoteException(message);
@@ -123,22 +120,18 @@ public class ReservationInputValidator {
       } else {
         minDuration += rr.getDuration();
       }
-      maxGangSize =
-          Resources.max(plan.getResourceCalculator(), plan.getTotalCapacity(),
-              maxGangSize,
-              Resources.multiply(rr.getCapability(), rr.getConcurrency()));
+      maxGangSize = Resources.max(plan.getResourceCalculator(),
+          plan.getTotalCapacity(), maxGangSize,
+          Resources.multiply(rr.getCapability(), rr.getConcurrency()));
     }
     // verify the allocation is possible (skip for ANY)
     long duration = contract.getDeadline() - contract.getArrival();
-    if (duration < minDuration
-        && type != ReservationRequestInterpreter.R_ANY) {
-      message =
-          "The time difference ("
-              + (duration)
-              + ") between arrival (" + contract.getArrival() + ") "
-              + "and deadline (" + contract.getDeadline() + ") must "
-              + " be greater or equal to the minimum resource duration ("
-              + minDuration + ")";
+    if (duration < minDuration && type != ReservationRequestInterpreter.R_ANY) {
+      message = "The time difference (" + (duration) + ") between arrival ("
+          + contract.getArrival() + ") " + "and deadline ("
+          + contract.getDeadline() + ") must "
+          + " be greater or equal to the minimum resource duration ("
+          + minDuration + ")";
       RMAuditLogger.logFailure("UNKNOWN", auditConstant,
           "validate reservation input definition", "ClientRMService", message);
       throw RPCUtil.getRemoteException(message);
@@ -148,10 +141,9 @@ public class ReservationInputValidator {
     if (Resources.greaterThan(plan.getResourceCalculator(),
         plan.getTotalCapacity(), maxGangSize, plan.getTotalCapacity())
         && type != ReservationRequestInterpreter.R_ANY) {
-      message =
-          "The size of the largest gang in the reservation definition ("
-              + maxGangSize + ") exceed the capacity available ("
-              + plan.getTotalCapacity() + " )";
+      message = "The size of the largest gang in the reservation definition ("
+          + maxGangSize + ") exceed the capacity available ("
+          + plan.getTotalCapacity() + " )";
       RMAuditLogger.logFailure("UNKNOWN", auditConstant,
           "validate reservation input definition", "ClientRMService", message);
       throw RPCUtil.getRemoteException(message);
@@ -179,32 +171,32 @@ public class ReservationInputValidator {
     }
   }
 
-  private Plan getPlanFromQueue(ReservationSystem reservationSystem, String
-          queue, String auditConstant) throws YarnException {
+  private Plan getPlanFromQueue(ReservationSystem reservationSystem,
+      String queue, String auditConstant) throws YarnException {
     String nullQueueErrorMessage = "The queue is not specified."
-            + " Please try again with a valid reservable queue.";
+        + " Please try again with a valid reservable queue.";
     String nullPlanErrorMessage = "The specified queue: " + queue
-            + " is not managed by reservation system."
-            + " Please try again with a valid reservable queue.";
+        + " is not managed by reservation system."
+        + " Please try again with a valid reservable queue.";
     return getPlanFromQueue(reservationSystem, queue, auditConstant,
-            nullQueueErrorMessage, nullPlanErrorMessage);
+        nullQueueErrorMessage, nullPlanErrorMessage);
   }
 
-  private Plan getPlanFromQueue(ReservationSystem reservationSystem, String
-          queue, String auditConstant, String nullQueueErrorMessage,
-          String nullPlanErrorMessage) throws YarnException {
+  private Plan getPlanFromQueue(ReservationSystem reservationSystem,
+      String queue, String auditConstant, String nullQueueErrorMessage,
+      String nullPlanErrorMessage) throws YarnException {
     if (queue == null || queue.isEmpty()) {
       RMAuditLogger.logFailure("UNKNOWN", auditConstant,
-              "validate reservation input", "ClientRMService",
-              nullQueueErrorMessage);
+          "validate reservation input", "ClientRMService",
+          nullQueueErrorMessage);
       throw RPCUtil.getRemoteException(nullQueueErrorMessage);
     }
     // check if the associated plan is valid
     Plan plan = reservationSystem.getPlan(queue);
     if (plan == null) {
       RMAuditLogger.logFailure("UNKNOWN", auditConstant,
-              "validate reservation input", "ClientRMService",
-              nullPlanErrorMessage);
+          "validate reservation input", "ClientRMService",
+          nullPlanErrorMessage);
       throw RPCUtil.getRemoteException(nullPlanErrorMessage);
     }
     return plan;
@@ -222,22 +214,21 @@ public class ReservationInputValidator {
    * @param reservationId the {@link ReservationId} associated with the current
    *          request
    * @return the {@link Plan} to submit the request to
-   * @throws YarnException
+   * @throws YarnException if validation fails
    */
   public Plan validateReservationSubmissionRequest(
-      ReservationSystem reservationSystem,
-      ReservationSubmissionRequest request, ReservationId reservationId)
-      throws YarnException {
+      ReservationSystem reservationSystem, ReservationSubmissionRequest request,
+      ReservationId reservationId) throws YarnException {
     String message;
     if (reservationId == null) {
-      message = "Reservation id cannot be null. Please try again " +
-        "specifying a valid reservation id by creating a new reservation id.";
+      message = "Reservation id cannot be null. Please try again specifying "
+          + " a valid reservation id by creating a new reservation id.";
       throw RPCUtil.getRemoteException(message);
     }
     // Check if it is a managed queue
     String queue = request.getQueue();
     Plan plan = getPlanFromQueue(reservationSystem, queue,
-            AuditConstants.SUBMIT_RESERVATION_REQUEST);
+        AuditConstants.SUBMIT_RESERVATION_REQUEST);
 
     validateReservationDefinition(reservationId,
         request.getReservationDefinition(), plan,
@@ -255,15 +246,14 @@ public class ReservationInputValidator {
    * @param request the {@link ReservationUpdateRequest} defining the resources
    *          required over time for the request
    * @return the {@link Plan} to submit the request to
-   * @throws YarnException
+   * @throws YarnException if validation fails
    */
   public Plan validateReservationUpdateRequest(
       ReservationSystem reservationSystem, ReservationUpdateRequest request)
       throws YarnException {
     ReservationId reservationId = request.getReservationId();
-    Plan plan =
-        validateReservation(reservationSystem, reservationId,
-            AuditConstants.UPDATE_RESERVATION_REQUEST);
+    Plan plan = validateReservation(reservationSystem, reservationId,
+        AuditConstants.UPDATE_RESERVATION_REQUEST);
     validateReservationDefinition(reservationId,
         request.getReservationDefinition(), plan,
         AuditConstants.UPDATE_RESERVATION_REQUEST);
@@ -278,28 +268,26 @@ public class ReservationInputValidator {
    *
    * @param reservationSystem the {@link ReservationSystem} to validate against
    * @param request the {@link ReservationListRequest} defining search
-   *                parameters for reservations in the {@link ReservationSystem}
-   *                that is being validated against.
+   *          parameters for reservations in the {@link ReservationSystem} that
+   *          is being validated against.
    * @return the {@link Plan} to list reservations of.
-   * @throws YarnException
+   * @throws YarnException if validation fails
    */
   public Plan validateReservationListRequest(
-      ReservationSystem reservationSystem,
-      ReservationListRequest request)
+      ReservationSystem reservationSystem, ReservationListRequest request)
       throws YarnException {
     String queue = request.getQueue();
     if (request.getEndTime() < request.getStartTime()) {
-      String errorMessage = "The specified end time must be greater than " +
-              "the specified start time.";
+      String errorMessage = "The specified end time must be greater than "
+          + "the specified start time.";
       RMAuditLogger.logFailure("UNKNOWN",
-              AuditConstants.LIST_RESERVATION_REQUEST,
-              "validate list reservation input", "ClientRMService",
-              errorMessage);
+          AuditConstants.LIST_RESERVATION_REQUEST,
+          "validate list reservation input", "ClientRMService", errorMessage);
       throw RPCUtil.getRemoteException(errorMessage);
     }
     // Check if it is a managed queue
     return getPlanFromQueue(reservationSystem, queue,
-            AuditConstants.LIST_RESERVATION_REQUEST);
+        AuditConstants.LIST_RESERVATION_REQUEST);
   }
 
   /**
@@ -312,7 +300,7 @@ public class ReservationInputValidator {
    * @param request the {@link ReservationDeleteRequest} defining the resources
    *          required over time for the request
    * @return the {@link Plan} to submit the request to
-   * @throws YarnException
+   * @throws YarnException if validation fails
    */
   public Plan validateReservationDeleteRequest(
       ReservationSystem reservationSystem, ReservationDeleteRequest request)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b6e7d136/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/ReservationSystem.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/reservation/ReservationSystem.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/ReservationSystem.java
index 8b62972..a6c8fcf 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/ReservationSystem.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/ReservationSystem.java
@@ -18,6 +18,8 @@
 
 package org.apache.hadoop.yarn.server.resourcemanager.reservation;
 
+import java.util.Map;
+
 import org.apache.hadoop.classification.InterfaceAudience.LimitedPrivate;
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.conf.Configuration;
@@ -29,8 +31,6 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Queue;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.security.ReservationsACLsManager;
 
-import java.util.Map;
-
 /**
  * This interface is the one implemented by any system that wants to support
  * Reservations i.e. make {@code Resource} allocations in future. Implementors
@@ -57,7 +57,7 @@ public interface ReservationSystem extends Recoverable {
    * 
    * @param conf configuration
    * @param rmContext current context of the {@code ResourceManager}
-   * @throws YarnException
+   * @throws YarnException if initialization of the configured plan fails
    */
   void reinitialize(Configuration conf, RMContext rmContext)
       throws YarnException;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b6e7d136/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/SharingPolicy.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/reservation/SharingPolicy.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/SharingPolicy.java
index e458055..cbf0f38 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/SharingPolicy.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/SharingPolicy.java
@@ -38,7 +38,7 @@ public interface SharingPolicy {
    * @param planQueuePath the name of the queue for this plan
    * @param conf the system configuration
    */
-  public void init(String planQueuePath, ReservationSchedulerConfiguration conf);
+  void init(String planQueuePath, ReservationSchedulerConfiguration conf);
 
   /**
    * This method runs the policy validation logic, and return true/false on
@@ -51,7 +51,7 @@ public interface SharingPolicy {
    * @throws PlanningException if the policy is respected if we add this
    *           {@link ReservationAllocation} to the {@link Plan}
    */
-  public void validate(Plan plan, ReservationAllocation newAllocation)
+  void validate(Plan plan, ReservationAllocation newAllocation)
       throws PlanningException;
 
   /**
@@ -68,9 +68,13 @@ public interface SharingPolicy {
    * @param start the start time for the range we are querying
    * @param end the end time for the range we are querying
    * @param oldId (optional) the id of a reservation being updated
+   *
+   * @return the available resources expressed as a
+   *         {@link RLESparseResourceAllocation}
+   *
    * @throws PlanningException throws if the request is not valid
    */
-  public RLESparseResourceAllocation availableResources(
+  RLESparseResourceAllocation availableResources(
       RLESparseResourceAllocation available, Plan plan, String user,
       ReservationId oldId, long start, long end) throws PlanningException;
 
@@ -82,7 +86,6 @@ public interface SharingPolicy {
    * 
    * @return validWindow the window of validity considered by the policy.
    */
-  public long getValidWindow();
-
+  long getValidWindow();
 
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b6e7d136/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/planning/Planner.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/reservation/planning/Planner.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/planning/Planner.java
index abac6ac..af0e712 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/planning/Planner.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/planning/Planner.java
@@ -34,7 +34,7 @@ public interface Planner {
    *
    * @param plan the {@link Plan} to replan
    * @param contracts the list of reservation requests
-   * @throws PlanningException
+   * @throws PlanningException if operation is unsuccessful
    */
   public void plan(Plan plan, List<ReservationDefinition> contracts)
       throws PlanningException;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b6e7d136/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/planning/PlanningAlgorithm.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/reservation/planning/PlanningAlgorithm.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/planning/PlanningAlgorithm.java
index 199bfa5..bbbf0d6 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/planning/PlanningAlgorithm.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/planning/PlanningAlgorithm.java
@@ -50,7 +50,7 @@ public abstract class PlanningAlgorithm implements ReservationAgent {
    * @return whether the allocateUser function was successful or not
    *
    * @throws PlanningException if the session cannot be fitted into the plan
-   * @throws ContractValidationException
+   * @throws ContractValidationException if validation fails
    */
   protected boolean allocateUser(ReservationId reservationId, String user,
       Plan plan, ReservationDefinition contract,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b6e7d136/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/planning/StageAllocator.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/reservation/planning/StageAllocator.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/planning/StageAllocator.java
index ec6d9c0..8934b0f 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/planning/StageAllocator.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/planning/StageAllocator.java
@@ -50,7 +50,7 @@ public interface StageAllocator {
    *
    * @return The computed allocation (or null if the stage could not be
    *         allocated)
-   * @throws PlanningException
+   * @throws PlanningException if operation is unsuccessful
    */
   Map<ReservationInterval, Resource> computeStageAllocation(Plan plan,
       RLESparseResourceAllocation planLoads,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b6e7d136/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/planning/StageAllocatorGreedy.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/reservation/planning/StageAllocatorGreedy.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/planning/StageAllocatorGreedy.java
index da04336..d107487 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/planning/StageAllocatorGreedy.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/planning/StageAllocatorGreedy.java
@@ -69,7 +69,7 @@ public class StageAllocatorGreedy implements StageAllocator {
 
     RLESparseResourceAllocation netAvailable =
         plan.getAvailableResourceOverTime(user, oldId, stageEarliestStart,
-            stageDeadline);
+            stageDeadline, 0);
 
     netAvailable =
         RLESparseResourceAllocation.merge(plan.getResourceCalculator(),

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b6e7d136/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/planning/StageAllocatorGreedyRLE.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/reservation/planning/StageAllocatorGreedyRLE.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/planning/StageAllocatorGreedyRLE.java
index ec83e02..ae7d91a 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/planning/StageAllocatorGreedyRLE.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/planning/StageAllocatorGreedyRLE.java
@@ -83,9 +83,8 @@ public class StageAllocatorGreedyRLE implements StageAllocator {
     int gangsToPlace = rr.getNumContainers() / rr.getConcurrency();
 
     // get available resources from plan
-    RLESparseResourceAllocation netRLERes =
-        plan.getAvailableResourceOverTime(user, oldId, stageEarliestStart,
-            stageDeadline);
+    RLESparseResourceAllocation netRLERes = plan.getAvailableResourceOverTime(
+        user, oldId, stageEarliestStart, stageDeadline, 0);
 
     // remove plan modifications
     netRLERes =

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b6e7d136/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/planning/StageAllocatorLowCostAligned.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/reservation/planning/StageAllocatorLowCostAligned.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/planning/StageAllocatorLowCostAligned.java
index e45f58c..c014549 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/planning/StageAllocatorLowCostAligned.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/planning/StageAllocatorLowCostAligned.java
@@ -77,8 +77,8 @@ public class StageAllocatorLowCostAligned implements StageAllocator {
     ResourceCalculator resCalc = plan.getResourceCalculator();
     Resource capacity = plan.getTotalCapacity();
 
-    RLESparseResourceAllocation netRLERes = plan
-        .getAvailableResourceOverTime(user, oldId, stageArrival, stageDeadline);
+    RLESparseResourceAllocation netRLERes = plan.getAvailableResourceOverTime(
+        user, oldId, stageArrival, stageDeadline, 0);
 
     long step = plan.getStep();
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b6e7d136/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/ReservationSystemTestUtil.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/reservation/ReservationSystemTestUtil.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/ReservationSystemTestUtil.java
index e99842e..5337e06 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/ReservationSystemTestUtil.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/ReservationSystemTestUtil.java
@@ -19,7 +19,10 @@ package org.apache.hadoop.yarn.server.resourcemanager.reservation;
 
 import static org.mockito.Matchers.any;
 import static org.mockito.Matchers.anySetOf;
-import static org.mockito.Mockito.*;
+import static org.mockito.Mockito.doReturn;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.spy;
+import static org.mockito.Mockito.when;
 
 import java.io.FileWriter;
 import java.io.IOException;
@@ -76,7 +79,8 @@ public class ReservationSystemTestUtil {
       String reservationQ, long timeWindow, float instConstraint,
       float avgConstraint) {
 
-    ReservationSchedulerConfiguration realConf = new CapacitySchedulerConfiguration();
+    ReservationSchedulerConfiguration realConf =
+        new CapacitySchedulerConfiguration();
     ReservationSchedulerConfiguration conf = spy(realConf);
     when(conf.getReservationWindow(reservationQ)).thenReturn(timeWindow);
     when(conf.getInstantaneousMaxCapacity(reservationQ))
@@ -168,7 +172,6 @@ public class ReservationSystemTestUtil {
     scheduler.start();
     scheduler.reinitialize(conf, rmContext);
 
-
     Resource resource =
         ReservationSystemTestUtil.calculateClusterResource(numContainers);
     RMNode node1 = MockNodes.newNodeInfo(1, resource, 1, "127.0.0.1");
@@ -184,10 +187,16 @@ public class ReservationSystemTestUtil {
 
   public static ReservationDefinition createSimpleReservationDefinition(
       long arrival, long deadline, long duration, int parallelism) {
+    return createSimpleReservationDefinition(arrival, deadline, duration,
+        parallelism, null);
+  }
+
+  public static ReservationDefinition createSimpleReservationDefinition(
+      long arrival, long deadline, long duration, int parallelism,
+      String recurrenceExpression) {
     // create a request with a single atomic ask
-    ReservationRequest r =
-        ReservationRequest.newInstance(Resource.newInstance(1024, 1),
-            parallelism, parallelism, duration);
+    ReservationRequest r = ReservationRequest.newInstance(
+        Resource.newInstance(1024, 1), parallelism, parallelism, duration);
     ReservationDefinition rDef = new ReservationDefinitionPBImpl();
     ReservationRequests reqs = new ReservationRequestsPBImpl();
     reqs.setReservationResources(Collections.singletonList(r));
@@ -195,32 +204,31 @@ public class ReservationSystemTestUtil {
     rDef.setReservationRequests(reqs);
     rDef.setArrival(arrival);
     rDef.setDeadline(deadline);
+    if (recurrenceExpression != null) {
+      rDef.setRecurrenceExpression(recurrenceExpression);
+    }
     return rDef;
   }
 
   public static ReservationSubmissionRequest createSimpleReservationRequest(
       ReservationId reservationId, int numContainers, long arrival,
       long deadline, long duration) {
-    return createSimpleReservationRequest(reservationId, numContainers,
-        arrival, deadline, duration, Priority.UNDEFINED);
+    return createSimpleReservationRequest(reservationId, numContainers, arrival,
+        deadline, duration, Priority.UNDEFINED);
   }
 
   public static ReservationSubmissionRequest createSimpleReservationRequest(
       ReservationId reservationId, int numContainers, long arrival,
       long deadline, long duration, Priority priority) {
     // create a request with a single atomic ask
-    ReservationRequest r =
-        ReservationRequest.newInstance(Resource.newInstance(1024, 1),
-            numContainers, 1, duration);
-    ReservationRequests reqs =
-        ReservationRequests.newInstance(Collections.singletonList(r),
-            ReservationRequestInterpreter.R_ALL);
-    ReservationDefinition rDef =
-        ReservationDefinition.newInstance(arrival, deadline, reqs,
-            "testClientRMService#reservation", "0", priority);
-    ReservationSubmissionRequest request =
-        ReservationSubmissionRequest.newInstance(rDef,
-            reservationQ, reservationId);
+    ReservationRequest r = ReservationRequest
+        .newInstance(Resource.newInstance(1024, 1), numContainers, 1, duration);
+    ReservationRequests reqs = ReservationRequests.newInstance(
+        Collections.singletonList(r), ReservationRequestInterpreter.R_ALL);
+    ReservationDefinition rDef = ReservationDefinition.newInstance(arrival,
+        deadline, reqs, "testClientRMService#reservation", "0", priority);
+    ReservationSubmissionRequest request = ReservationSubmissionRequest
+        .newInstance(rDef, reservationQ, reservationId);
     return request;
   }
 
@@ -252,9 +260,9 @@ public class ReservationSystemTestUtil {
     return cs;
   }
 
-  @SuppressWarnings("rawtypes") public static void initializeRMContext(
-      int numContainers, AbstractYarnScheduler scheduler,
-      RMContext mockRMContext) {
+  @SuppressWarnings("rawtypes")
+  public static void initializeRMContext(int numContainers,
+      AbstractYarnScheduler scheduler, RMContext mockRMContext) {
 
     when(mockRMContext.getScheduler()).thenReturn(scheduler);
     Resource r = calculateClusterResource(numContainers);
@@ -262,26 +270,25 @@ public class ReservationSystemTestUtil {
   }
 
   public static RMContext createRMContext(Configuration conf) {
-    RMContext mockRmContext = Mockito.spy(
-        new RMContextImpl(null, null, null, null, null, null,
-            new RMContainerTokenSecretManager(conf),
-            new NMTokenSecretManagerInRM(conf),
-            new ClientToAMTokenSecretManagerInRM(), null));
+    RMContext mockRmContext = Mockito.spy(new RMContextImpl(null, null, null,
+        null, null, null, new RMContainerTokenSecretManager(conf),
+        new NMTokenSecretManagerInRM(conf),
+        new ClientToAMTokenSecretManagerInRM(), null));
 
     RMNodeLabelsManager nlm = mock(RMNodeLabelsManager.class);
     when(nlm.getQueueResource(any(String.class), anySetOf(String.class),
-            any(Resource.class))).thenAnswer(new Answer<Resource>() {
-      @Override public Resource answer(InvocationOnMock invocation)
-          throws Throwable {
-        Object[] args = invocation.getArguments();
-        return (Resource) args[2];
-      }
-    });
+        any(Resource.class))).thenAnswer(new Answer<Resource>() {
+          @Override
+          public Resource answer(InvocationOnMock invocation) throws Throwable {
+            Object[] args = invocation.getArguments();
+            return (Resource) args[2];
+          }
+        });
 
     when(nlm.getResourceByLabel(any(String.class), any(Resource.class)))
         .thenAnswer(new Answer<Resource>() {
-          @Override public Resource answer(InvocationOnMock invocation)
-              throws Throwable {
+          @Override
+          public Resource answer(InvocationOnMock invocation) throws Throwable {
             Object[] args = invocation.getArguments();
             return (Resource) args[1];
           }
@@ -304,9 +311,8 @@ public class ReservationSystemTestUtil {
     final String A = CapacitySchedulerConfiguration.ROOT + ".a";
     conf.setCapacity(A, 10);
 
-    final String dedicated =
-        CapacitySchedulerConfiguration.ROOT + CapacitySchedulerConfiguration.DOT
-            + reservationQ;
+    final String dedicated = CapacitySchedulerConfiguration.ROOT
+        + CapacitySchedulerConfiguration.DOT + reservationQ;
     conf.setCapacity(dedicated, 80);
     // Set as reservation queue
     conf.setReservable(dedicated, true);
@@ -405,26 +411,55 @@ public class ReservationSystemTestUtil {
 
   public static Map<ReservationInterval, Resource> generateAllocation(
       long startTime, long step, int[] alloc) {
+    return generateAllocation(startTime, step, alloc, null);
+  }
+
+  public static Map<ReservationInterval, Resource> generateAllocation(
+      long startTime, long step, int[] alloc, String recurrenceExpression) {
     Map<ReservationInterval, Resource> req = new TreeMap<>();
-    for (int i = 0; i < alloc.length; i++) {
-      req.put(new ReservationInterval(startTime + i * step,
-          startTime + (i + 1) * step), ReservationSystemUtil.toResource(
-          ReservationRequest
-              .newInstance(Resource.newInstance(1024, 1), alloc[i])));
+
+    long period = 0;
+    if (recurrenceExpression != null) {
+      period = Long.parseLong(recurrenceExpression);
+    }
+
+    long rStart;
+    long rEnd;
+    for (int j = 0; j < 86400000; j += period) {
+      for (int i = 0; i < alloc.length; i++) {
+        rStart = (startTime + i * step) + j * period;
+        rEnd = (startTime + (i + 1) * step) + j * period;
+        if (period > 0) {
+          rStart = rStart % period + j * period;
+          rEnd = rEnd % period + j * period;
+          if (rStart > rEnd) {
+            // skip wrap-around entry
+            continue;
+          }
+        }
+
+        req.put(new ReservationInterval(rStart, rEnd),
+            ReservationSystemUtil.toResource(ReservationRequest
+                .newInstance(Resource.newInstance(1024, 1), alloc[i])));
+
+      }
+      // execute only once if non-periodic
+      if (period == 0) {
+        break;
+      }
     }
     return req;
   }
 
-  public static RLESparseResourceAllocation
-      generateRLESparseResourceAllocation(int[] alloc, long[] timeSteps) {
+  public static RLESparseResourceAllocation generateRLESparseResourceAllocation(
+      int[] alloc, long[] timeSteps) {
     TreeMap<Long, Resource> allocationsMap = new TreeMap<>();
     for (int i = 0; i < alloc.length; i++) {
       allocationsMap.put(timeSteps[i],
           Resource.newInstance(alloc[i], alloc[i]));
     }
-    RLESparseResourceAllocation rleVector =
-        new RLESparseResourceAllocation(allocationsMap,
-            new DefaultResourceCalculator());
+    RLESparseResourceAllocation rleVector = new RLESparseResourceAllocation(
+        allocationsMap, new DefaultResourceCalculator());
     return rleVector;
   }
 


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[09/37] hadoop git commit: HADOOP-14820 Wasb mkdirs security checks inconsistent with HDFS. Contributed by Sivaguru Sankaridurg

Posted by ae...@apache.org.
HADOOP-14820 Wasb mkdirs security checks inconsistent with HDFS.
Contributed by Sivaguru Sankaridurg


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

Branch: refs/heads/HDFS-7240
Commit: 792eff9ea70da2c6e0ff5a1b177a51e7b2fb96eb
Parents: 5dba545
Author: Steve Loughran <st...@apache.org>
Authored: Tue Sep 5 14:16:57 2017 +0100
Committer: Steve Loughran <st...@apache.org>
Committed: Tue Sep 5 14:16:57 2017 +0100

----------------------------------------------------------------------
 .../hadoop/fs/azure/NativeAzureFileSystem.java  |  9 ++-
 .../TestNativeAzureFileSystemAuthorization.java | 64 ++++++++++++++++++++
 .../TestAzureFileSystemInstrumentation.java     | 18 +++---
 3 files changed, 77 insertions(+), 14 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/792eff9e/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/NativeAzureFileSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/NativeAzureFileSystem.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/NativeAzureFileSystem.java
index 2abc6c6..0bde124 100644
--- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/NativeAzureFileSystem.java
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/NativeAzureFileSystem.java
@@ -2425,13 +2425,13 @@ public class NativeAzureFileSystem extends FileSystem {
 
   private Path getAncestor(Path f) throws IOException {
 
-    for (Path current = f.getParent(), parent = current.getParent();
+    for (Path current = f, parent = current.getParent();
          parent != null; // Stop when you get to the root
          current = parent, parent = current.getParent()) {
 
       String currentKey = pathToKey(current);
       FileMetadata currentMetadata = store.retrieveMetadata(currentKey);
-      if (currentMetadata != null) {
+      if (currentMetadata != null && currentMetadata.isDir()) {
         Path ancestor = keyToPath(currentMetadata.getKey());
         LOG.debug("Found ancestor {}, for path: {}", ancestor.toString(), f.toString());
         return ancestor;
@@ -2448,7 +2448,6 @@ public class NativeAzureFileSystem extends FileSystem {
 
   public boolean mkdirs(Path f, FsPermission permission, boolean noUmask) throws IOException {
 
-
     LOG.debug("Creating directory: {}", f.toString());
 
     if (containsColon(f)) {
@@ -2459,6 +2458,10 @@ public class NativeAzureFileSystem extends FileSystem {
     Path absolutePath = makeAbsolute(f);
     Path ancestor = getAncestor(absolutePath);
 
+    if (absolutePath.equals(ancestor)) {
+      return true;
+    }
+
     performAuthCheck(ancestor, WasbAuthorizationOperations.WRITE, "mkdirs", absolutePath);
 
     PermissionStatus permissionStatus = null;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/792eff9e/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestNativeAzureFileSystemAuthorization.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestNativeAzureFileSystemAuthorization.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestNativeAzureFileSystemAuthorization.java
index 91d6ebb..a3f2843 100644
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestNativeAzureFileSystemAuthorization.java
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestNativeAzureFileSystemAuthorization.java
@@ -604,6 +604,70 @@ public class TestNativeAzureFileSystemAuthorization
   }
 
   /**
+   * Positive test for mkdirs -p with existing hierarchy
+   * @throws Throwable
+   */
+  @Test
+  public void testMkdirsWithExistingHierarchyCheckPositive1() throws Throwable {
+
+    Path testPath = new Path("/testMkdirsWithExistingHierarchyCheckPositive1");
+
+    authorizer.addAuthRule("/", WasbAuthorizationOperations.WRITE.toString(), true);
+    fs.updateWasbAuthorizer(authorizer);
+
+    try {
+      fs.mkdirs(testPath);
+      ContractTestUtils.assertIsDirectory(fs, testPath);
+
+      /* Don't need permissions to create a directory that already exists */
+      authorizer.deleteAllAuthRules();
+
+      fs.mkdirs(testPath);
+      ContractTestUtils.assertIsDirectory(fs, testPath);
+    }
+    finally {
+      allowRecursiveDelete(fs, testPath.toString());
+      fs.delete(testPath, true);
+    }
+  }
+
+  @Test
+  public void testMkdirsWithExistingHierarchyCheckPositive2() throws Throwable {
+
+    Path testPath = new Path("/testMkdirsWithExistingHierarchyCheckPositive2");
+    Path childPath1 = new Path(testPath, "1");
+    Path childPath2 = new Path(childPath1, "2");
+    Path childPath3 = new Path(childPath2, "3");
+
+    authorizer.addAuthRule("/",
+        WasbAuthorizationOperations.WRITE.toString(), true);
+
+    authorizer.addAuthRule(childPath1.toString(),
+        WasbAuthorizationOperations.WRITE.toString(), true);
+
+    fs.updateWasbAuthorizer(authorizer);
+
+    try {
+      fs.mkdirs(childPath1);
+      ContractTestUtils.assertIsDirectory(fs, childPath1);
+
+      // Path already exists => no-op.
+      fs.mkdirs(testPath);
+      ContractTestUtils.assertIsDirectory(fs, testPath);
+
+      // Path already exists => no-op.
+      fs.mkdirs(childPath1);
+      ContractTestUtils.assertIsDirectory(fs, childPath1);
+
+      // Check permissions against existing ancestor childPath1
+      fs.mkdirs(childPath3);
+      ContractTestUtils.assertIsDirectory(fs, childPath3);
+    } finally {
+      allowRecursiveDelete(fs, testPath.toString());
+      fs.delete(testPath, true);
+    }
+  }
+  /**
    * Negative test for mkdirs access check
    * @throws Throwable
    */

http://git-wip-us.apache.org/repos/asf/hadoop/blob/792eff9e/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/metrics/TestAzureFileSystemInstrumentation.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/metrics/TestAzureFileSystemInstrumentation.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/metrics/TestAzureFileSystemInstrumentation.java
index a1a021b..818a844 100644
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/metrics/TestAzureFileSystemInstrumentation.java
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/metrics/TestAzureFileSystemInstrumentation.java
@@ -108,17 +108,13 @@ public class TestAzureFileSystemInstrumentation {
 
     // Create a directory
     assertTrue(fs.mkdirs(new Path("a")));
-    // At the time of writing, it takes 1 request to create the actual directory,
-    // plus 2 requests per level to check that there's no blob with that name and
-    // 1 request per level above to create it if it doesn't exist.
-    // So for the path above (/user/<name>/a), it takes 2 requests each to check
-    // there's no blob called /user, no blob called /user/<name> and no blob
-    // called /user/<name>/a, and then 3 request for the creation of the three
-    // levels, and then 2 requests for checking/stamping the version of AS,
-    // totaling 11.
-    // Also, there's the initial 1 request for container check so total is 12.
-    // The getAncestor call at the very beginning adds another 4 calls, totalling 16.
-    base = assertWebResponsesInRange(base, 1, 16);
+    // At the time of writing
+    // getAncestor uses 2 calls for each folder level /user/<name>/a
+    // plus 1 call made by checkContainer
+    // mkdir checks the hierarchy with 2 calls per level
+    // mkdirs calls storeEmptyDir to create the empty folder, which makes 5 calls
+    // For a total of 7 + 6 + 5 = 18 web responses
+    base = assertWebResponsesInRange(base, 1, 18);
     assertEquals(1,
         AzureMetricsTestUtil.getLongCounterValue(getInstrumentation(), WASB_DIRECTORIES_CREATED));
 


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[06/37] hadoop git commit: YARN-7152. [ATSv2] Registering timeline client before AMRMClient service init throw exception (Rohith Sharma K S via Varun Saxena)

Posted by ae...@apache.org.
YARN-7152. [ATSv2] Registering timeline client before AMRMClient service init throw exception (Rohith Sharma K S via Varun Saxena)


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

Branch: refs/heads/HDFS-7240
Commit: ef87d34a0e96f6b6390b7086c4ea2d5b7a4c2754
Parents: 633c1ea
Author: Varun Saxena <va...@apache.org>
Authored: Mon Sep 4 20:11:02 2017 +0530
Committer: Varun Saxena <va...@apache.org>
Committed: Mon Sep 4 20:11:02 2017 +0530

----------------------------------------------------------------------
 .../hadoop/yarn/client/api/AMRMClient.java      | 22 ++------------------
 1 file changed, 2 insertions(+), 20 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/ef87d34a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/AMRMClient.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/AMRMClient.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/AMRMClient.java
index 60e305f..b11c094 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/AMRMClient.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/AMRMClient.java
@@ -28,7 +28,6 @@ import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.classification.InterfaceAudience.Public;
 import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.service.AbstractService;
 import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterResponse;
@@ -42,7 +41,6 @@ import org.apache.hadoop.yarn.api.records.Priority;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.UpdateContainerRequest;
 import org.apache.hadoop.yarn.client.api.impl.AMRMClientImpl;
-import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.util.resource.Resources;
 
@@ -57,7 +55,6 @@ public abstract class AMRMClient<T extends AMRMClient.ContainerRequest> extends
   private static final Log LOG = LogFactory.getLog(AMRMClient.class);
 
   private TimelineV2Client timelineV2Client;
-  private boolean timelineServiceV2Enabled;
 
   /**
    * Create a new instance of AMRMClient.
@@ -82,12 +79,6 @@ public abstract class AMRMClient<T extends AMRMClient.ContainerRequest> extends
     nmTokenCache = NMTokenCache.getSingleton();
   }
 
-  @Override
-  protected void serviceInit(Configuration conf) throws Exception {
-    super.serviceInit(conf);
-    timelineServiceV2Enabled = YarnConfiguration.timelineServiceV2Enabled(conf);
-  }
-
   /**
    * Object to represent a single container request for resources. Scheduler
    * documentation should be consulted for the specifics of how the parameters
@@ -696,18 +687,9 @@ public abstract class AMRMClient<T extends AMRMClient.ContainerRequest> extends
    * V2 client will be updated dynamically if registered.
    *
    * @param client the timeline v2 client to register
-   * @throws YarnException when this method is invoked even when ATS V2 is not
-   *           configured.
    */
-  public void registerTimelineV2Client(TimelineV2Client client)
-      throws YarnException {
-    if (timelineServiceV2Enabled) {
-      timelineV2Client = client;
-    } else {
-      LOG.error("Trying to register timeline v2 client when not configured.");
-      throw new YarnException(
-          "register timeline v2 client when not configured.");
-    }
+  public void registerTimelineV2Client(TimelineV2Client client) {
+    timelineV2Client = client;
   }
 
   /**


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[17/37] hadoop git commit: HADOOP-12077. Provide a multi-URI replication Inode for ViewFs. Contributed by Gera Shegalov

Posted by ae...@apache.org.
HADOOP-12077. Provide a multi-URI replication Inode for ViewFs. Contributed by Gera Shegalov


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

Branch: refs/heads/HDFS-7240
Commit: 1f3bc63e6772be81bc9a6a7d93ed81d2a9e066c0
Parents: 63720ef
Author: Chris Douglas <cd...@apache.org>
Authored: Tue Sep 5 23:30:18 2017 -0700
Committer: Chris Douglas <cd...@apache.org>
Committed: Tue Sep 5 23:51:51 2017 -0700

----------------------------------------------------------------------
 .../org/apache/hadoop/fs/viewfs/ConfigUtil.java |  27 +
 .../org/apache/hadoop/fs/viewfs/Constants.java  |   8 +-
 .../org/apache/hadoop/fs/viewfs/InodeTree.java  |  64 +-
 .../apache/hadoop/fs/viewfs/NflyFSystem.java    | 951 +++++++++++++++++++
 .../apache/hadoop/fs/viewfs/ViewFileSystem.java |  37 +-
 .../org/apache/hadoop/fs/viewfs/ViewFs.java     |  10 +-
 .../TestViewFileSystemLocalFileSystem.java      |  77 +-
 .../hadoop/fs/viewfs/TestViewFsConfig.java      |  13 +-
 .../fs/viewfs/TestViewFileSystemHdfs.java       | 151 ++-
 9 files changed, 1275 insertions(+), 63 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/1f3bc63e/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ConfigUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ConfigUtil.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ConfigUtil.java
index 6900df2..a5fc62e 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ConfigUtil.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ConfigUtil.java
@@ -20,6 +20,7 @@ package org.apache.hadoop.fs.viewfs;
 import java.net.URI;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.util.StringUtils;
 
 /**
  * Utilities for config variables of the viewFs See {@link ViewFs}
@@ -69,6 +70,32 @@ public class ConfigUtil {
   }
   
   /**
+   *
+   * @param conf
+   * @param mountTableName
+   * @param src
+   * @param settings
+   * @param targets
+   */
+  public static void addLinkNfly(Configuration conf, String mountTableName,
+      String src, String settings, final URI ... targets) {
+
+    settings = settings == null
+        ? "minReplication=2,repairOnRead=true"
+        : settings;
+
+    conf.set(getConfigViewFsPrefix(mountTableName) + "." +
+            Constants.CONFIG_VIEWFS_LINK_NFLY + "." + settings + "." + src,
+        StringUtils.uriToString(targets));
+  }
+
+  public static void addLinkNfly(final Configuration conf, final String src,
+      final URI ... targets) {
+    addLinkNfly(conf, Constants.CONFIG_VIEWFS_DEFAULT_MOUNT_TABLE, src, null,
+        targets);
+  }
+
+  /**
    * Add config variable for homedir for default mount table
    * @param conf - add to this conf
    * @param homedir - the home dir path starting with slash

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1f3bc63e/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/Constants.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/Constants.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/Constants.java
index 9882a8e..1a07c10 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/Constants.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/Constants.java
@@ -57,7 +57,13 @@ public interface Constants {
    * Config variable for specifying a merge link
    */
   public static final String CONFIG_VIEWFS_LINK_MERGE = "linkMerge";
-  
+
+  /**
+   * Config variable for specifying an nfly link. Nfly writes to multiple
+   * locations, and allows reads from the closest one.
+   */
+  String CONFIG_VIEWFS_LINK_NFLY = "linkNfly";
+
   /**
    * Config variable for specifying a merge of the root of the mount-table
    *  with the root of another file system. 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1f3bc63e/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/InodeTree.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/InodeTree.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/InodeTree.java
index c62d5cc..665c9c9 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/InodeTree.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/InodeTree.java
@@ -133,6 +133,12 @@ abstract class InodeTree<T> {
     }
   }
 
+  enum LinkType {
+    SINGLE,
+    MERGE,
+    NFLY
+  }
+
   /**
    * An internal class to represent a mount link.
    * A mount link can be single dir link or a merge dir link.
@@ -146,19 +152,17 @@ abstract class InodeTree<T> {
    * is changed later it is then ignored (a dir with null entries)
    */
   static class INodeLink<T> extends INode<T> {
-    final boolean isMergeLink; // true if MergeLink
     final URI[] targetDirLinkList;
     final T targetFileSystem;   // file system object created from the link.
 
     /**
-     * Construct a mergeLink.
+     * Construct a mergeLink or nfly.
      */
     INodeLink(final String pathToNode, final UserGroupInformation aUgi,
         final T targetMergeFs, final URI[] aTargetDirLinkList) {
       super(pathToNode, aUgi);
       targetFileSystem = targetMergeFs;
       targetDirLinkList = aTargetDirLinkList;
-      isMergeLink = true;
     }
 
     /**
@@ -170,7 +174,6 @@ abstract class InodeTree<T> {
       targetFileSystem = targetFs;
       targetDirLinkList = new URI[1];
       targetDirLinkList[0] = aTargetDirLink;
-      isMergeLink = false;
     }
 
     /**
@@ -188,7 +191,9 @@ abstract class InodeTree<T> {
   }
 
   private void createLink(final String src, final String target,
-      final boolean isLinkMerge, final UserGroupInformation aUgi)
+      final LinkType linkType, final String settings,
+      final UserGroupInformation aUgi,
+      final Configuration config)
       throws URISyntaxException, IOException,
       FileAlreadyExistsException, UnsupportedFileSystemException {
     // Validate that src is valid absolute path
@@ -235,18 +240,20 @@ abstract class InodeTree<T> {
     final INodeLink<T> newLink;
     final String fullPath = curInode.fullPath + (curInode == root ? "" : "/")
         + iPath;
-    if (isLinkMerge) { // Target is list of URIs
-      String[] targetsList = StringUtils.getStrings(target);
-      URI[] targetsListURI = new URI[targetsList.length];
-      int k = 0;
-      for (String itarget : targetsList) {
-        targetsListURI[k++] = new URI(itarget);
-      }
-      newLink = new INodeLink<T>(fullPath, aUgi,
-          getTargetFileSystem(targetsListURI), targetsListURI);
-    } else {
+    switch (linkType) {
+    case SINGLE:
       newLink = new INodeLink<T>(fullPath, aUgi,
           getTargetFileSystem(new URI(target)), new URI(target));
+      break;
+    case MERGE:
+    case NFLY:
+      final URI[] targetUris = StringUtils.stringToURI(
+          StringUtils.getStrings(target));
+      newLink = new INodeLink<T>(fullPath, aUgi,
+            getTargetFileSystem(settings, targetUris), targetUris);
+      break;
+    default:
+      throw new IllegalArgumentException(linkType + ": Infeasible linkType");
     }
     curInode.addLink(iPath, newLink);
     mountPoints.add(new MountPoint<T>(src, newLink));
@@ -257,14 +264,14 @@ abstract class InodeTree<T> {
    * 3 abstract methods.
    * @throws IOException
    */
-  protected abstract T getTargetFileSystem(final URI uri)
+  protected abstract T getTargetFileSystem(URI uri)
       throws UnsupportedFileSystemException, URISyntaxException, IOException;
 
-  protected abstract T getTargetFileSystem(final INodeDir<T> dir)
+  protected abstract T getTargetFileSystem(INodeDir<T> dir)
       throws URISyntaxException;
 
-  protected abstract T getTargetFileSystem(final URI[] mergeFsURIList)
-      throws UnsupportedFileSystemException, URISyntaxException;
+  protected abstract T getTargetFileSystem(String settings, URI[] mergeFsURIs)
+      throws UnsupportedFileSystemException, URISyntaxException, IOException;
 
   /**
    * Create Inode Tree from the specified mount-table specified in Config
@@ -298,8 +305,9 @@ abstract class InodeTree<T> {
       final String key = si.getKey();
       if (key.startsWith(mtPrefix)) {
         gotMountTableEntry = true;
-        boolean isMergeLink = false;
+        LinkType linkType = LinkType.SINGLE;
         String src = key.substring(mtPrefix.length());
+        String settings = null;
         if (src.startsWith(linkPrefix)) {
           src = src.substring(linkPrefix.length());
           if (src.equals(SlashPath.toString())) {
@@ -309,8 +317,20 @@ abstract class InodeTree<T> {
                 + "supported yet.");
           }
         } else if (src.startsWith(linkMergePrefix)) { // A merge link
-          isMergeLink = true;
+          linkType = LinkType.MERGE;
           src = src.substring(linkMergePrefix.length());
+        } else if (src.startsWith(Constants.CONFIG_VIEWFS_LINK_NFLY)) {
+          // prefix.settings.src
+          src = src.substring(Constants.CONFIG_VIEWFS_LINK_NFLY.length() + 1);
+          // settings.src
+          settings = src.substring(0, src.indexOf('.'));
+          // settings
+
+          // settings.src
+          src = src.substring(settings.length() + 1);
+          // src
+
+          linkType = LinkType.NFLY;
         } else if (src.startsWith(Constants.CONFIG_VIEWFS_HOMEDIR)) {
           // ignore - we set home dir from config
           continue;
@@ -319,7 +339,7 @@ abstract class InodeTree<T> {
               "Mount table in config: " + src);
         }
         final String target = si.getValue(); // link or merge link
-        createLink(src, target, isMergeLink, ugi);
+        createLink(src, target, linkType, settings, ugi, config);
       }
     }
     if (!gotMountTableEntry) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1f3bc63e/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/NflyFSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/NflyFSystem.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/NflyFSystem.java
new file mode 100644
index 0000000..53966b8
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/NflyFSystem.java
@@ -0,0 +1,951 @@
+/**
+ * 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.fs.viewfs;
+
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CommonConfigurationKeys;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.FileUtil;
+import org.apache.hadoop.fs.LocatedFileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.RemoteIterator;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.io.MultipleIOException;
+import org.apache.hadoop.net.DNSToSwitchMapping;
+import org.apache.hadoop.net.NetworkTopology;
+import org.apache.hadoop.net.Node;
+import org.apache.hadoop.net.NodeBase;
+import org.apache.hadoop.net.ScriptBasedMapping;
+import org.apache.hadoop.util.Progressable;
+import org.apache.hadoop.util.ReflectionUtils;
+import org.apache.hadoop.util.StringUtils;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.net.InetAddress;
+import java.net.URI;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.BitSet;
+import java.util.EnumSet;
+import java.util.Iterator;
+import java.util.List;
+
+/**
+ * Nfly is a multi filesystem mount point.
+ */
+@Private
+final class NflyFSystem extends FileSystem {
+  private static final Log LOG = LogFactory.getLog(NflyFSystem.class);
+  private static final String NFLY_TMP_PREFIX = "_nfly_tmp_";
+
+  enum NflyKey {
+    // minimum replication, if local filesystem is included +1 is recommended
+    minReplication,
+
+    // forces to check all the replicas and fetch the one with the most recent
+    // time stamp
+    //
+    readMostRecent,
+
+    // create missing replica from far to near, including local?
+    repairOnRead
+  }
+
+  private static final int DEFAULT_MIN_REPLICATION = 2;
+  private static URI nflyURI = URI.create("nfly:///");
+
+  private final NflyNode[] nodes;
+  private final int minReplication;
+  private final EnumSet<NflyKey> nflyFlags;
+  private final Node myNode;
+  private final NetworkTopology topology;
+
+  /**
+   * URI's authority is used as an approximation of the distance from the
+   * client. It's sufficient for DC but not accurate because worker nodes can be
+   * closer.
+   */
+  private static class NflyNode extends NodeBase {
+    private final ChRootedFileSystem fs;
+    NflyNode(String hostName, String rackName, URI uri,
+        Configuration conf) throws IOException {
+      this(hostName, rackName, new ChRootedFileSystem(uri, conf));
+    }
+
+    NflyNode(String hostName, String rackName, ChRootedFileSystem fs) {
+      super(hostName, rackName);
+      this.fs = fs;
+    }
+
+    ChRootedFileSystem getFs() {
+      return fs;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+      // satisfy findbugs
+      return super.equals(o);
+    }
+
+    @Override
+    public int hashCode() {
+      // satisfy findbugs
+      return super.hashCode();
+    }
+
+  }
+
+  private static final class MRNflyNode
+      extends NflyNode implements Comparable<MRNflyNode> {
+
+    private FileStatus status;
+
+    private MRNflyNode(NflyNode n) {
+      super(n.getName(), n.getNetworkLocation(), n.fs);
+    }
+
+    private void updateFileStatus(Path f) throws IOException {
+      final FileStatus tmpStatus = getFs().getFileStatus(f);
+      status = tmpStatus == null
+          ? notFoundStatus(f)
+          : tmpStatus;
+    }
+
+    // TODO allow configurable error margin for FileSystems with different
+    // timestamp precisions
+    @Override
+    public int compareTo(MRNflyNode other) {
+      if (status == null) {
+        return other.status == null ? 0 : 1; // move non-null towards head
+      } else if (other.status == null) {
+        return -1; // move this towards head
+      } else {
+        final long mtime = status.getModificationTime();
+        final long their = other.status.getModificationTime();
+        return Long.compare(their, mtime); // move more recent towards head
+      }
+    }
+
+    @Override
+    public boolean equals(Object o) {
+      if (!(o instanceof MRNflyNode)) {
+        return false;
+      }
+      MRNflyNode other = (MRNflyNode) o;
+      return 0 == compareTo(other);
+    }
+
+    @Override
+    public int hashCode() {
+      // satisfy findbugs
+      return super.hashCode();
+    }
+
+    private FileStatus nflyStatus() throws IOException {
+      return new NflyStatus(getFs(), status);
+    }
+
+    private FileStatus cloneStatus() throws IOException {
+      return new FileStatus(status.getLen(),
+          status.isDirectory(),
+          status.getReplication(),
+          status.getBlockSize(),
+          status.getModificationTime(),
+          status.getAccessTime(),
+          null, null, null,
+          status.isSymlink() ? status.getSymlink() : null,
+          status.getPath());
+    }
+  }
+
+  private MRNflyNode[] workSet() {
+    final MRNflyNode[] res = new MRNflyNode[nodes.length];
+    for (int i = 0; i < res.length; i++) {
+      res[i] = new MRNflyNode(nodes[i]);
+    }
+    return res;
+  }
+
+
+  /**
+   * Utility to replace null with DEFAULT_RACK.
+   *
+   * @param rackString rack value, can be null
+   * @return non-null rack string
+   */
+  private static String getRack(String rackString) {
+    return rackString == null ? NetworkTopology.DEFAULT_RACK : rackString;
+  }
+
+  /**
+   * Creates a new Nfly instance.
+   *
+   * @param uris the list of uris in the mount point
+   * @param conf configuration object
+   * @param minReplication minimum copies to commit a write op
+   * @param nflyFlags modes such readMostRecent
+   * @throws IOException
+   */
+  private NflyFSystem(URI[] uris, Configuration conf, int minReplication,
+      EnumSet<NflyKey> nflyFlags) throws IOException {
+    if (uris.length < minReplication) {
+      throw new IOException(minReplication + " < " + uris.length
+          + ": Minimum replication < #destinations");
+    }
+    setConf(conf);
+    final String localHostName = InetAddress.getLocalHost().getHostName();
+
+    // build a list for topology resolution
+    final List<String> hostStrings = new ArrayList<String>(uris.length + 1);
+    for (URI uri : uris) {
+      final String uriHost = uri.getHost();
+      // assume local file system or another closest filesystem if no authority
+      hostStrings.add(uriHost == null ? localHostName : uriHost);
+    }
+    // resolve the client node
+    hostStrings.add(localHostName);
+
+    final DNSToSwitchMapping tmpDns = ReflectionUtils.newInstance(conf.getClass(
+        CommonConfigurationKeys.NET_TOPOLOGY_NODE_SWITCH_MAPPING_IMPL_KEY,
+        ScriptBasedMapping.class, DNSToSwitchMapping.class), conf);
+
+    // this is an ArrayList
+    final List<String> rackStrings = tmpDns.resolve(hostStrings);
+    nodes = new NflyNode[uris.length];
+    final Iterator<String> rackIter = rackStrings.iterator();
+    for (int i = 0; i < nodes.length; i++) {
+      nodes[i] = new NflyNode(hostStrings.get(i), rackIter.next(), uris[i],
+          conf);
+    }
+    // sort all the uri's by distance from myNode, the local file system will
+    // automatically be the the first one.
+    //
+    myNode = new NodeBase(localHostName, getRack(rackIter.next()));
+    topology = NetworkTopology.getInstance(conf);
+    topology.sortByDistance(myNode, nodes, nodes.length);
+
+    this.minReplication = minReplication;
+    this.nflyFlags = nflyFlags;
+    statistics = getStatistics(nflyURI.getScheme(), getClass());
+  }
+
+  /**
+   * Transactional output stream. When creating path /dir/file
+   * 1) create invisible /real/dir_i/_nfly_tmp_file
+   * 2) when more than min replication was written, write is committed by
+   *   renaming all successfully written files to /real/dir_i/file
+   */
+  private final class NflyOutputStream extends OutputStream {
+    // actual path
+    private final Path nflyPath;
+    // tmp path before commit
+    private final Path tmpPath;
+    // broadcast set
+    private final FSDataOutputStream[] outputStreams;
+    // status set: 1 working, 0 problem
+    private final BitSet opSet;
+    private final boolean useOverwrite;
+
+    private NflyOutputStream(Path f, FsPermission permission, boolean overwrite,
+        int bufferSize, short replication, long blockSize,
+        Progressable progress) throws IOException {
+      nflyPath = f;
+      tmpPath = getNflyTmpPath(f);
+      outputStreams = new FSDataOutputStream[nodes.length];
+      for (int i = 0; i < outputStreams.length; i++) {
+        outputStreams[i] = nodes[i].fs.create(tmpPath, permission, true,
+            bufferSize, replication, blockSize, progress);
+      }
+      opSet = new BitSet(outputStreams.length);
+      opSet.set(0, outputStreams.length);
+      useOverwrite = false;
+    }
+
+    //
+    // TODO consider how to clean up and throw an exception early when the clear
+    // bits under min replication
+    //
+
+    private void mayThrow(List<IOException> ioExceptions) throws IOException {
+      final IOException ioe = MultipleIOException
+          .createIOException(ioExceptions);
+      if (opSet.cardinality() < minReplication) {
+        throw ioe;
+      } else {
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("Exceptions occurred: " + ioe);
+        }
+      }
+    }
+
+
+    @Override
+    public void write(int d) throws IOException {
+      final List<IOException> ioExceptions = new ArrayList<IOException>();
+      for (int i = opSet.nextSetBit(0);
+           i >=0;
+           i = opSet.nextSetBit(i + 1)) {
+        try {
+          outputStreams[i].write(d);
+        } catch (Throwable t) {
+          osException(i, "write", t, ioExceptions);
+        }
+      }
+      mayThrow(ioExceptions);
+    }
+
+    private void osException(int i, String op, Throwable t,
+        List<IOException> ioExceptions) {
+      opSet.clear(i);
+      processThrowable(nodes[i], op, t, ioExceptions, tmpPath, nflyPath);
+    }
+
+    @Override
+    public void write(byte[] bytes, int offset, int len) throws IOException {
+      final List<IOException> ioExceptions = new ArrayList<IOException>();
+      for (int i = opSet.nextSetBit(0);
+           i >= 0;
+           i = opSet.nextSetBit(i + 1)) {
+        try {
+          outputStreams[i].write(bytes, offset, len);
+        } catch (Throwable t) {
+          osException(i, "write", t, ioExceptions);
+        }
+      }
+      mayThrow(ioExceptions);
+    }
+
+    @Override
+    public void flush() throws IOException {
+      final List<IOException> ioExceptions = new ArrayList<IOException>();
+      for (int i = opSet.nextSetBit(0);
+           i >= 0;
+           i = opSet.nextSetBit(i + 1)) {
+        try {
+          outputStreams[i].flush();
+        } catch (Throwable t) {
+          osException(i, "flush", t, ioExceptions);
+        }
+      }
+      mayThrow(ioExceptions);
+    }
+
+    @Override
+    public void close() throws IOException {
+      final List<IOException> ioExceptions = new ArrayList<IOException>();
+      for (int i = opSet.nextSetBit(0);
+           i >= 0;
+           i = opSet.nextSetBit(i + 1)) {
+        try {
+          outputStreams[i].close();
+        } catch (Throwable t) {
+          osException(i, "close", t, ioExceptions);
+        }
+      }
+      if (opSet.cardinality() < minReplication) {
+        cleanupAllTmpFiles();
+        throw new IOException("Failed to sufficiently replicate: min="
+            + minReplication + " actual=" + opSet.cardinality());
+      } else {
+        commit();
+      }
+    }
+
+    private void cleanupAllTmpFiles() throws IOException {
+      for (int i = 0; i < outputStreams.length; i++) {
+        try {
+          nodes[i].fs.delete(tmpPath);
+        } catch (Throwable t) {
+          processThrowable(nodes[i], "delete", t, null, tmpPath);
+        }
+      }
+    }
+
+    private void commit() throws IOException {
+      final List<IOException> ioExceptions = new ArrayList<IOException>();
+      for (int i = opSet.nextSetBit(0);
+           i >= 0;
+           i = opSet.nextSetBit(i + 1)) {
+        final NflyNode nflyNode = nodes[i];
+        try {
+          if (useOverwrite) {
+            nflyNode.fs.delete(nflyPath);
+          }
+          nflyNode.fs.rename(tmpPath, nflyPath);
+
+        } catch (Throwable t) {
+          osException(i, "commit", t, ioExceptions);
+        }
+      }
+
+      if (opSet.cardinality() < minReplication) {
+        // cleanup should be done outside. If rename failed, it's unlikely that
+        // delete will work either. It's the same kind of metadata-only op
+        //
+        throw MultipleIOException.createIOException(ioExceptions);
+      }
+
+      // best effort to have a consistent timestamp
+      final long commitTime = System.currentTimeMillis();
+      for (int i = opSet.nextSetBit(0);
+          i >= 0;
+          i = opSet.nextSetBit(i + 1)) {
+        try {
+          nodes[i].fs.setTimes(nflyPath, commitTime, commitTime);
+        } catch (Throwable t) {
+          LOG.info("Failed to set timestamp: " + nodes[i] + " " + nflyPath);
+        }
+      }
+    }
+  }
+
+  private Path getNflyTmpPath(Path f) {
+    return new Path(f.getParent(), NFLY_TMP_PREFIX + f.getName());
+  }
+
+  /**
+   * // TODO
+   * Some file status implementations have expensive deserialization or metadata
+   * retrieval. This probably does not go beyond RawLocalFileSystem. Wrapping
+   * the the real file status to preserve this behavior. Otherwise, calling
+   * realStatus getters in constructor defeats this design.
+   */
+  static final class NflyStatus extends FileStatus {
+    private static final long serialVersionUID = 0x21f276d8;
+
+    private final FileStatus realStatus;
+    private final String strippedRoot;
+
+    private NflyStatus(ChRootedFileSystem realFs, FileStatus realStatus)
+        throws IOException {
+      this.realStatus = realStatus;
+      this.strippedRoot = realFs.stripOutRoot(realStatus.getPath());
+    }
+
+    String stripRoot() throws IOException {
+      return strippedRoot;
+    }
+
+    @Override
+    public long getLen() {
+      return realStatus.getLen();
+    }
+
+    @Override
+    public boolean isFile() {
+      return realStatus.isFile();
+    }
+
+    @Override
+    public boolean isDirectory() {
+      return realStatus.isDirectory();
+    }
+
+    @Override
+    public boolean isSymlink() {
+      return realStatus.isSymlink();
+    }
+
+    @Override
+    public long getBlockSize() {
+      return realStatus.getBlockSize();
+    }
+
+    @Override
+    public short getReplication() {
+      return realStatus.getReplication();
+    }
+
+    @Override
+    public long getModificationTime() {
+      return realStatus.getModificationTime();
+    }
+
+    @Override
+    public long getAccessTime() {
+      return realStatus.getAccessTime();
+    }
+
+    @Override
+    public FsPermission getPermission() {
+      return realStatus.getPermission();
+    }
+
+    @Override
+    public String getOwner() {
+      return realStatus.getOwner();
+    }
+
+    @Override
+    public String getGroup() {
+      return realStatus.getGroup();
+    }
+
+    @Override
+    public Path getPath() {
+      return realStatus.getPath();
+    }
+
+    @Override
+    public void setPath(Path p) {
+      realStatus.setPath(p);
+    }
+
+    @Override
+    public Path getSymlink() throws IOException {
+      return realStatus.getSymlink();
+    }
+
+    @Override
+    public void setSymlink(Path p) {
+      realStatus.setSymlink(p);
+    }
+
+    @Override
+    public boolean equals(Object o) {
+      return realStatus.equals(o);
+    }
+
+    @Override
+    public int hashCode() {
+      return realStatus.hashCode();
+    }
+
+    @Override
+    public String toString() {
+      return realStatus.toString();
+    }
+  }
+
+  @Override
+  public URI getUri() {
+    return nflyURI;
+  }
+
+  /**
+   * Category: READ.
+   *
+   * @param f the file name to open
+   * @param bufferSize the size of the buffer to be used.
+   * @return input stream according to nfly flags (closest, most recent)
+   * @throws IOException
+   * @throws FileNotFoundException iff all destinations generate this exception
+   */
+  @Override
+  public FSDataInputStream open(Path f, int bufferSize) throws IOException {
+    // TODO proxy stream for reads
+    final List<IOException> ioExceptions =
+        new ArrayList<IOException>(nodes.length);
+    int numNotFounds = 0;
+    final MRNflyNode[] mrNodes = workSet();
+
+    // naively iterate until one can be opened
+    //
+    for (final MRNflyNode nflyNode : mrNodes) {
+      try {
+        if (nflyFlags.contains(NflyKey.repairOnRead)
+            || nflyFlags.contains(NflyKey.readMostRecent)) {
+          // calling file status to avoid pulling bytes prematurely
+          nflyNode.updateFileStatus(f);
+        } else {
+          return nflyNode.getFs().open(f, bufferSize);
+        }
+      } catch (FileNotFoundException fnfe) {
+        nflyNode.status = notFoundStatus(f);
+        numNotFounds++;
+        processThrowable(nflyNode, "open", fnfe, ioExceptions, f);
+      } catch (Throwable t) {
+        processThrowable(nflyNode, "open", t, ioExceptions, f);
+      }
+    }
+
+    if (nflyFlags.contains(NflyKey.readMostRecent)) {
+      // sort from most recent to least recent
+      Arrays.sort(mrNodes);
+    }
+
+    final FSDataInputStream fsdisAfterRepair = repairAndOpen(mrNodes, f,
+        bufferSize);
+
+    if (fsdisAfterRepair != null) {
+      return fsdisAfterRepair;
+    }
+
+    mayThrowFileNotFound(ioExceptions, numNotFounds);
+    throw MultipleIOException.createIOException(ioExceptions);
+  }
+
+  private static FileStatus notFoundStatus(Path f) {
+    return new FileStatus(-1, false, 0, 0, 0, f);
+  }
+
+  /**
+   * Iterate all available nodes in the proximity order to attempt repair of all
+   * FileNotFound nodes.
+   *
+   * @param mrNodes work set copy of nodes
+   * @param f path to repair and open
+   * @param bufferSize buffer size for read RPC
+   * @return the closest/most recent replica stream AFTER repair
+   */
+  private FSDataInputStream repairAndOpen(MRNflyNode[] mrNodes, Path f,
+      int bufferSize) {
+    long maxMtime = 0L;
+    for (final MRNflyNode srcNode : mrNodes) {
+      if (srcNode.status == null  // not available
+          || srcNode.status.getLen() < 0L) { // not found
+        continue; // not available
+      }
+      if (srcNode.status.getModificationTime() > maxMtime) {
+        maxMtime = srcNode.status.getModificationTime();
+      }
+
+      // attempt to repair all notFound nodes with srcNode
+      //
+      for (final MRNflyNode dstNode : mrNodes) {
+        if (dstNode.status == null // not available
+            || srcNode.compareTo(dstNode) == 0) { // same mtime
+          continue;
+        }
+
+        try {
+          // status is absolute from the underlying mount, making it chrooted
+          //
+          final FileStatus srcStatus = srcNode.cloneStatus();
+          srcStatus.setPath(f);
+          final Path tmpPath = getNflyTmpPath(f);
+          FileUtil.copy(srcNode.getFs(), srcStatus, dstNode.getFs(), tmpPath,
+              false,                // don't delete
+              true,                 // overwrite
+              getConf());
+          dstNode.getFs().delete(f, false);
+          if (dstNode.getFs().rename(tmpPath, f)) {
+            try {
+              dstNode.getFs().setTimes(f, srcNode.status.getModificationTime(),
+                  srcNode.status.getAccessTime());
+            } finally {
+              // save getFileStatus rpc
+              srcStatus.setPath(dstNode.getFs().makeQualified(f));
+              dstNode.status = srcStatus;
+            }
+          }
+        } catch (IOException ioe) {
+          // can blame the source by statusSet.clear(ai), however, it would
+          // cost an extra RPC, so just rely on the loop below that will attempt
+          // an open anyhow
+          //
+          LOG.info(f + " " + srcNode + "->" + dstNode + ": Failed to repair",
+                ioe);
+        }
+      }
+    }
+
+    // Since Java7, QuickSort is used instead of MergeSort.
+    // QuickSort may not be stable and thus the equal most recent nodes, may no
+    // longer appear in the NetworkTopology order.
+    //
+    if (maxMtime > 0) {
+      final List<MRNflyNode> mrList = new ArrayList<MRNflyNode>();
+      for (final MRNflyNode openNode : mrNodes) {
+        if (openNode.status != null && openNode.status.getLen() >= 0L) {
+          if (openNode.status.getModificationTime() == maxMtime) {
+            mrList.add(openNode);
+          }
+        }
+      }
+      // assert mrList.size > 0
+      final MRNflyNode[] readNodes = mrList.toArray(new MRNflyNode[0]);
+      topology.sortByDistance(myNode, readNodes, readNodes.length);
+      for (final MRNflyNode rNode : readNodes) {
+        try {
+          return rNode.getFs().open(f, bufferSize);
+        } catch (IOException e) {
+          LOG.info(f + ": Failed to open at " + rNode.getFs().getUri());
+        }
+      }
+    }
+    return null;
+  }
+
+  private void mayThrowFileNotFound(List<IOException> ioExceptions,
+      int numNotFounds) throws FileNotFoundException {
+    if (numNotFounds == nodes.length) {
+      throw (FileNotFoundException)ioExceptions.get(nodes.length - 1);
+    }
+  }
+
+  // WRITE
+  @Override
+  public FSDataOutputStream create(Path f, FsPermission permission,
+      boolean overwrite, int bufferSize, short replication, long blockSize,
+      Progressable progress) throws IOException {
+    return new FSDataOutputStream(new NflyOutputStream(f, permission, overwrite,
+        bufferSize, replication, blockSize, progress), statistics);
+  }
+
+  // WRITE
+  @Override
+  public FSDataOutputStream append(Path f, int bufferSize,
+      Progressable progress) throws IOException {
+    return null;
+  }
+
+  // WRITE
+  @Override
+  public boolean rename(Path src, Path dst) throws IOException {
+    final List<IOException> ioExceptions = new ArrayList<IOException>();
+    int numNotFounds = 0;
+    boolean succ = true;
+    for (final NflyNode nflyNode : nodes) {
+      try {
+        succ &= nflyNode.fs.rename(src, dst);
+      } catch (FileNotFoundException fnfe) {
+        numNotFounds++;
+        processThrowable(nflyNode, "rename", fnfe, ioExceptions, src, dst);
+      } catch (Throwable t) {
+        processThrowable(nflyNode, "rename", t, ioExceptions, src, dst);
+        succ = false;
+      }
+    }
+
+    mayThrowFileNotFound(ioExceptions, numNotFounds);
+
+    // if all destinations threw exceptions throw, otherwise return
+    //
+    if (ioExceptions.size() == nodes.length) {
+      throw MultipleIOException.createIOException(ioExceptions);
+    }
+
+    return succ;
+  }
+
+  // WRITE
+  @Override
+  public boolean delete(Path f, boolean recursive) throws IOException {
+    final List<IOException> ioExceptions = new ArrayList<IOException>();
+    int numNotFounds = 0;
+    boolean succ = true;
+    for (final NflyNode nflyNode : nodes) {
+      try {
+        succ &= nflyNode.fs.delete(f);
+      } catch (FileNotFoundException fnfe) {
+        numNotFounds++;
+        processThrowable(nflyNode, "delete", fnfe, ioExceptions, f);
+      } catch (Throwable t) {
+        processThrowable(nflyNode, "delete", t, ioExceptions, f);
+        succ = false;
+      }
+    }
+    mayThrowFileNotFound(ioExceptions, numNotFounds);
+
+    // if all destinations threw exceptions throw, otherwise return
+    //
+    if (ioExceptions.size() == nodes.length) {
+      throw MultipleIOException.createIOException(ioExceptions);
+    }
+
+    return succ;
+  }
+
+
+  /**
+   * Returns the closest non-failing destination's result.
+   *
+   * @param f given path
+   * @return array of file statuses according to nfly modes
+   * @throws FileNotFoundException
+   * @throws IOException
+   */
+  @Override
+  public FileStatus[] listStatus(Path f) throws FileNotFoundException,
+      IOException {
+    final List<IOException> ioExceptions =
+        new ArrayList<IOException>(nodes.length);
+
+    final MRNflyNode[] mrNodes = workSet();
+    if (nflyFlags.contains(NflyKey.readMostRecent)) {
+      int numNotFounds = 0;
+      for (final MRNflyNode nflyNode : mrNodes) {
+        try {
+          nflyNode.updateFileStatus(f);
+        } catch (FileNotFoundException fnfe) {
+          numNotFounds++;
+          processThrowable(nflyNode, "listStatus", fnfe, ioExceptions, f);
+        } catch (Throwable t) {
+          processThrowable(nflyNode, "listStatus", t, ioExceptions, f);
+        }
+      }
+      mayThrowFileNotFound(ioExceptions, numNotFounds);
+      Arrays.sort(mrNodes);
+    }
+
+    int numNotFounds = 0;
+    for (final MRNflyNode nflyNode : mrNodes) {
+      try {
+        final FileStatus[] realStats = nflyNode.getFs().listStatus(f);
+        final FileStatus[] nflyStats = new FileStatus[realStats.length];
+        for (int i = 0; i < realStats.length; i++) {
+          nflyStats[i] = new NflyStatus(nflyNode.getFs(), realStats[i]);
+        }
+        return nflyStats;
+      } catch (FileNotFoundException fnfe) {
+        numNotFounds++;
+        processThrowable(nflyNode, "listStatus", fnfe, ioExceptions, f);
+      } catch (Throwable t) {
+        processThrowable(nflyNode, "listStatus", t, ioExceptions, f);
+      }
+    }
+    mayThrowFileNotFound(ioExceptions, numNotFounds);
+    throw MultipleIOException.createIOException(ioExceptions);
+  }
+
+  @Override
+  public RemoteIterator<LocatedFileStatus> listLocatedStatus(Path f)
+      throws FileNotFoundException, IOException {
+    // TODO important for splits
+    return super.listLocatedStatus(f);
+  }
+
+  @Override
+  public void setWorkingDirectory(Path newDir) {
+    for (final NflyNode nflyNode : nodes) {
+      nflyNode.fs.setWorkingDirectory(newDir);
+    }
+  }
+
+  @Override
+  public Path getWorkingDirectory() {
+    return nodes[0].fs.getWorkingDirectory(); // 0 is as good as any
+  }
+
+  @Override
+  public boolean mkdirs(Path f, FsPermission permission) throws IOException {
+    boolean succ = true;
+    for (final NflyNode nflyNode : nodes) {
+      succ &= nflyNode.fs.mkdirs(f, permission);
+    }
+    return succ;
+  }
+
+  @Override
+  public FileStatus getFileStatus(Path f) throws IOException {
+    // TODO proxy stream for reads
+    final List<IOException> ioExceptions =
+        new ArrayList<IOException>(nodes.length);
+    int numNotFounds = 0;
+    final MRNflyNode[] mrNodes = workSet();
+
+    long maxMtime = Long.MIN_VALUE;
+    int maxMtimeIdx = Integer.MIN_VALUE;
+
+    // naively iterate until one can be returned
+    //
+    for (int i = 0; i < mrNodes.length; i++) {
+      MRNflyNode nflyNode = mrNodes[i];
+      try {
+        nflyNode.updateFileStatus(f);
+        if (nflyFlags.contains(NflyKey.readMostRecent)) {
+          final long nflyTime = nflyNode.status.getModificationTime();
+          if (nflyTime > maxMtime) {
+            maxMtime = nflyTime;
+            maxMtimeIdx = i;
+          }
+        } else {
+          return nflyNode.nflyStatus();
+        }
+      } catch (FileNotFoundException fnfe) {
+        numNotFounds++;
+        processThrowable(nflyNode, "getFileStatus", fnfe, ioExceptions, f);
+      } catch (Throwable t) {
+        processThrowable(nflyNode, "getFileStatus", t, ioExceptions, f);
+      }
+    }
+
+    if (maxMtimeIdx >= 0) {
+      return mrNodes[maxMtimeIdx].nflyStatus();
+    }
+
+    mayThrowFileNotFound(ioExceptions, numNotFounds);
+    throw MultipleIOException.createIOException(ioExceptions);
+  }
+
+  private static void processThrowable(NflyNode nflyNode, String op,
+      Throwable t, List<IOException> ioExceptions,
+      Path... f) {
+    final String errMsg = Arrays.toString(f)
+        + ": failed to " + op + " " + nflyNode.fs.getUri();
+    final IOException ioex;
+    if (t instanceof FileNotFoundException) {
+      ioex = new FileNotFoundException(errMsg);
+      ioex.initCause(t);
+    } else {
+      ioex = new IOException(errMsg, t);
+    }
+
+    if (ioExceptions != null) {
+      ioExceptions.add(ioex);
+    }
+  }
+
+  /**
+   * Initializes an nfly mountpoint in viewfs.
+   *
+   * @param uris destinations to replicate writes to
+   * @param conf file system configuration
+   * @param settings comma-separated list of k=v pairs.
+   * @return an Nfly filesystem
+   * @throws IOException
+   */
+  static FileSystem createFileSystem(URI[] uris, Configuration conf,
+      String settings) throws IOException {
+    // assert settings != null
+    int minRepl = DEFAULT_MIN_REPLICATION;
+    EnumSet<NflyKey> nflyFlags = EnumSet.noneOf(NflyKey.class);
+    final String[] kvPairs = StringUtils.split(settings);
+    for (String kv : kvPairs) {
+      final String[] kvPair = StringUtils.split(kv, '=');
+      if (kvPair.length != 2) {
+        throw new IllegalArgumentException(kv);
+      }
+      NflyKey nflyKey = NflyKey.valueOf(kvPair[0]);
+      switch (nflyKey) {
+      case minReplication:
+        minRepl = Integer.parseInt(kvPair[1]);
+        break;
+      case repairOnRead:
+      case readMostRecent:
+        if (Boolean.valueOf(kvPair[1])) {
+          nflyFlags.add(nflyKey);
+        }
+        break;
+      default:
+        throw new IllegalArgumentException(nflyKey + ": Infeasible");
+      }
+    }
+    return new NflyFSystem(uris, conf, minRepl, nflyFlags);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1f3bc63e/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ViewFileSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ViewFileSystem.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ViewFileSystem.java
index 158b099..ca1380a 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ViewFileSystem.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ViewFileSystem.java
@@ -54,7 +54,6 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.PathFilter;
 import org.apache.hadoop.fs.QuotaUsage;
 import org.apache.hadoop.fs.RemoteIterator;
-import org.apache.hadoop.fs.UnsupportedFileSystemException;
 import org.apache.hadoop.fs.XAttrSetFlag;
 import org.apache.hadoop.fs.permission.AclEntry;
 import org.apache.hadoop.fs.permission.AclStatus;
@@ -186,25 +185,21 @@ public class ViewFileSystem extends FileSystem {
       fsState = new InodeTree<FileSystem>(conf, authority) {
 
         @Override
-        protected
-        FileSystem getTargetFileSystem(final URI uri)
+        protected FileSystem getTargetFileSystem(final URI uri)
           throws URISyntaxException, IOException {
             return new ChRootedFileSystem(uri, config);
         }
 
         @Override
-        protected
-        FileSystem getTargetFileSystem(final INodeDir<FileSystem> dir)
+        protected FileSystem getTargetFileSystem(final INodeDir<FileSystem> dir)
           throws URISyntaxException {
           return new InternalDirOfViewFs(dir, creationTime, ugi, myUri, config);
         }
 
         @Override
-        protected
-        FileSystem getTargetFileSystem(URI[] mergeFsURIList)
-            throws URISyntaxException, UnsupportedFileSystemException {
-          throw new UnsupportedFileSystemException("mergefs not implemented");
-          // return MergeFs.createMergeFs(mergeFsURIList, config);
+        protected FileSystem getTargetFileSystem(final String settings,
+            final URI[] uris) throws URISyntaxException, IOException {
+          return NflyFSystem.createFileSystem(uris, config, settings);
         }
       };
       workingDir = this.getHomeDirectory();
@@ -455,8 +450,13 @@ public class ViewFileSystem extends FileSystem {
 
   private Path getChrootedPath(InodeTree.ResolveResult<FileSystem> res,
       FileStatus status, Path f) throws IOException {
-    final String suffix = ((ChRootedFileSystem)res.targetFileSystem)
-        .stripOutRoot(status.getPath());
+    final String suffix;
+    if (res.targetFileSystem instanceof ChRootedFileSystem) {
+      suffix = ((ChRootedFileSystem)res.targetFileSystem)
+          .stripOutRoot(status.getPath());
+    } else { // nfly
+      suffix = ((NflyFSystem.NflyStatus)status).stripRoot();
+    }
     return this.makeQualified(
         suffix.length() == 0 ? f : new Path(res.resolvedPath, suffix));
   }
@@ -501,10 +501,15 @@ public class ViewFileSystem extends FileSystem {
     verifyRenameStrategy(srcUri, dstUri,
         resSrc.targetFileSystem == resDst.targetFileSystem, renameStrategy);
 
-    ChRootedFileSystem srcFS = (ChRootedFileSystem) resSrc.targetFileSystem;
-    ChRootedFileSystem dstFS = (ChRootedFileSystem) resDst.targetFileSystem;
-    return srcFS.getMyFs().rename(srcFS.fullPath(resSrc.remainingPath),
-        dstFS.fullPath(resDst.remainingPath));
+    if (resSrc.targetFileSystem instanceof ChRootedFileSystem &&
+        resDst.targetFileSystem instanceof ChRootedFileSystem) {
+      ChRootedFileSystem srcFS = (ChRootedFileSystem) resSrc.targetFileSystem;
+      ChRootedFileSystem dstFS = (ChRootedFileSystem) resDst.targetFileSystem;
+      return srcFS.getMyFs().rename(srcFS.fullPath(resSrc.remainingPath),
+          dstFS.fullPath(resDst.remainingPath));
+    } else {
+      return resSrc.targetFileSystem.rename(resSrc.remainingPath, resDst.remainingPath);
+    }
   }
 
   static void verifyRenameStrategy(URI srcUri, URI dstUri,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1f3bc63e/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ViewFs.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ViewFs.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ViewFs.java
index 364485f..6a89f27 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ViewFs.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ViewFs.java
@@ -212,8 +212,7 @@ public class ViewFs extends AbstractFileSystem {
     fsState = new InodeTree<AbstractFileSystem>(conf, authority) {
 
       @Override
-      protected
-      AbstractFileSystem getTargetFileSystem(final URI uri)
+      protected AbstractFileSystem getTargetFileSystem(final URI uri)
         throws URISyntaxException, UnsupportedFileSystemException {
           String pathString = uri.getPath();
           if (pathString.isEmpty()) {
@@ -225,15 +224,14 @@ public class ViewFs extends AbstractFileSystem {
       }
 
       @Override
-      protected
-      AbstractFileSystem getTargetFileSystem(
+      protected AbstractFileSystem getTargetFileSystem(
           final INodeDir<AbstractFileSystem> dir) throws URISyntaxException {
         return new InternalDirOfViewFs(dir, creationTime, ugi, getUri());
       }
 
       @Override
-      protected
-      AbstractFileSystem getTargetFileSystem(URI[] mergeFsURIList)
+      protected AbstractFileSystem getTargetFileSystem(final String settings,
+          final URI[] mergeFsURIList)
           throws URISyntaxException, UnsupportedFileSystemException {
         throw new UnsupportedFileSystemException("mergefs not implemented yet");
         // return MergeFs.createMergeFs(mergeFsURIList, config);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1f3bc63e/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFileSystemLocalFileSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFileSystemLocalFileSystem.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFileSystemLocalFileSystem.java
index 4943792..808d8b0 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFileSystemLocalFileSystem.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFileSystemLocalFileSystem.java
@@ -18,13 +18,25 @@
 package org.apache.hadoop.fs.viewfs;
 
 
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+import java.io.IOException;
+import java.net.URI;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.FileSystemTestHelper;
+import org.apache.hadoop.fs.Path;
 
 import org.junit.After;
 import org.junit.Before;
-
+import org.junit.Test;
 
 
 /**
@@ -37,6 +49,8 @@ import org.junit.Before;
  */
 
 public class TestViewFileSystemLocalFileSystem extends ViewFileSystemBaseTest {
+  private static final Log LOG =
+      LogFactory.getLog(TestViewFileSystemLocalFileSystem.class);
 
   @Override
   @Before
@@ -47,6 +61,65 @@ public class TestViewFileSystemLocalFileSystem extends ViewFileSystemBaseTest {
     
   }
 
+  @Test
+  public void testNflyWriteSimple() throws IOException {
+    LOG.info("Starting testNflyWriteSimple");
+    final URI[] testUris = new URI[] {
+        URI.create(targetTestRoot + "/nfwd1"),
+        URI.create(targetTestRoot + "/nfwd2")
+    };
+    final String testFileName = "test.txt";
+    final Configuration testConf = new Configuration(conf);
+    final String testString = "Hello Nfly!";
+    final Path nflyRoot = new Path("/nflyroot");
+    ConfigUtil.addLinkNfly(testConf, nflyRoot.toString(), testUris);
+    final FileSystem nfly = FileSystem.get(URI.create("viewfs:///"), testConf);
+
+    final FSDataOutputStream fsDos = nfly.create(
+        new Path(nflyRoot, "test.txt"));
+    try {
+      fsDos.writeUTF(testString);
+    } finally {
+      fsDos.close();
+    }
+
+    FileStatus[] statuses = nfly.listStatus(nflyRoot);
+
+    FileSystem lfs = FileSystem.getLocal(testConf);
+    for (final URI testUri : testUris) {
+      final Path testFile = new Path(new Path(testUri), testFileName);
+      assertTrue(testFile + " should exist!",  lfs.exists(testFile));
+      final FSDataInputStream fsdis = lfs.open(testFile);
+      try {
+        assertEquals("Wrong file content", testString, fsdis.readUTF());
+      } finally {
+        fsdis.close();
+      }
+    }
+  }
+
+
+  @Test
+  public void testNflyInvalidMinReplication() throws Exception {
+    LOG.info("Starting testNflyInvalidMinReplication");
+    final URI[] testUris = new URI[] {
+        URI.create(targetTestRoot + "/nfwd1"),
+        URI.create(targetTestRoot + "/nfwd2")
+    };
+
+    final Configuration conf = new Configuration();
+    ConfigUtil.addLinkNfly(conf, "mt", "/nflyroot", "minReplication=4",
+        testUris);
+    try {
+      FileSystem.get(URI.create("viewfs://mt/"), conf);
+      fail("Expected bad minReplication exception.");
+    } catch (IOException ioe) {
+      assertTrue("No minReplication message",
+          ioe.getMessage().contains("Minimum replication"));
+    }
+  }
+
+
   @Override
   @After
   public void tearDown() throws Exception {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1f3bc63e/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFsConfig.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFsConfig.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFsConfig.java
index 895ae0c..136837f 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFsConfig.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFsConfig.java
@@ -24,7 +24,6 @@ import java.net.URISyntaxException;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.FileAlreadyExistsException;
-import org.apache.hadoop.fs.UnsupportedFileSystemException;
 import org.junit.Test;
 
 public class TestViewFsConfig {
@@ -43,23 +42,21 @@ public class TestViewFsConfig {
     new InodeTree<Foo>(conf, null) {
 
       @Override
-      protected Foo getTargetFileSystem(final URI uri)
-          throws URISyntaxException, UnsupportedFileSystemException {
+      protected Foo getTargetFileSystem(final URI uri) {
         return null;
       }
 
       @Override
-      protected Foo getTargetFileSystem(
-          org.apache.hadoop.fs.viewfs.InodeTree.INodeDir<Foo> dir)
-          throws URISyntaxException {
+      protected Foo getTargetFileSystem(final INodeDir<Foo> dir) {
         return null;
       }
 
       @Override
-      protected Foo getTargetFileSystem(URI[] mergeFsURIList)
-          throws URISyntaxException, UnsupportedFileSystemException {
+      protected Foo getTargetFileSystem(final String settings,
+          final URI[] mergeFsURIList) {
         return null;
       }
+
     };
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1f3bc63e/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFileSystemHdfs.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFileSystemHdfs.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFileSystemHdfs.java
index b8f5379..b8bed1d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFileSystemHdfs.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFileSystemHdfs.java
@@ -17,11 +17,9 @@
  */
 package org.apache.hadoop.fs.viewfs;
 
-
-import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.FS_TRASH_INTERVAL_KEY;
-
 import java.io.File;
 import java.io.IOException;
+import java.net.URI;
 import java.net.URISyntaxException;
 import java.util.EnumSet;
 
@@ -31,6 +29,8 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.crypto.key.JavaKeyStoreProvider;
 import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileChecksum;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
@@ -45,17 +45,26 @@ import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.MiniDFSNNTopology;
 import org.apache.hadoop.hdfs.client.CreateEncryptionZoneFlag;
 import org.apache.hadoop.hdfs.client.HdfsAdmin;
+import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.test.GenericTestUtils;
+import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.FS_TRASH_INTERVAL_KEY;
+import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.IPC_CLIENT_CONNECT_MAX_RETRIES_KEY;
+
 import org.junit.After;
 import org.junit.AfterClass;
-import org.junit.Assert;
 import org.junit.Before;
 import org.junit.BeforeClass;
 import org.junit.Test;
+import static org.junit.Assert.*;
 
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 public class TestViewFileSystemHdfs extends ViewFileSystemBaseTest {
+  private static final Logger LOG =
+      LoggerFactory.getLogger(TestViewFileSystemHdfs.class);
+
 
   private static MiniDFSCluster cluster;
   private static Path defaultWorkingDirectory;
@@ -190,12 +199,12 @@ public class TestViewFileSystemHdfs extends ViewFileSystemBaseTest {
 
     //Verify file deletion within EZ
     DFSTestUtil.verifyDelete(shell, fsTarget, encFile, true);
-    Assert.assertTrue("ViewFileSystem trash roots should include EZ file trash",
+    assertTrue("ViewFileSystem trash roots should include EZ file trash",
         (fsView.getTrashRoots(true).size() == 1));
 
     //Verify deletion of EZ
     DFSTestUtil.verifyDelete(shell, fsTarget, zone, true);
-    Assert.assertTrue("ViewFileSystem trash roots should include EZ zone trash",
+    assertTrue("ViewFileSystem trash roots should include EZ zone trash",
         (fsView.getTrashRoots(true).size() == 2));
   }
 
@@ -239,14 +248,14 @@ public class TestViewFileSystemHdfs extends ViewFileSystemBaseTest {
         viewFs.getFileChecksum(mountDataFilePath);
     FileChecksum fileChecksumViaTargetFs =
         fsTarget.getFileChecksum(fsTargetFilePath);
-    Assert.assertTrue("File checksum not matching!",
+    assertTrue("File checksum not matching!",
         fileChecksumViaViewFs.equals(fileChecksumViaTargetFs));
 
     fileChecksumViaViewFs =
         viewFs.getFileChecksum(mountDataFilePath, fileLength / 2);
     fileChecksumViaTargetFs =
         fsTarget.getFileChecksum(fsTargetFilePath, fileLength / 2);
-    Assert.assertTrue("File checksum not matching!",
+    assertTrue("File checksum not matching!",
         fileChecksumViaViewFs.equals(fileChecksumViaTargetFs));
   }
 
@@ -269,4 +278,130 @@ public class TestViewFileSystemHdfs extends ViewFileSystemBaseTest {
               e);
     }
   }
+
+  @Test
+  public void testNflyClosestRepair() throws Exception {
+    testNflyRepair(NflyFSystem.NflyKey.repairOnRead);
+  }
+
+  @Test
+  public void testNflyMostRecentRepair() throws Exception {
+    testNflyRepair(NflyFSystem.NflyKey.readMostRecent);
+  }
+
+  private void testNflyRepair(NflyFSystem.NflyKey repairKey)
+      throws Exception {
+    LOG.info("Starting testNflyWriteSimpleFailover");
+    final URI uri1 = targetTestRoot.toUri();
+    final URI uri2 = targetTestRoot2.toUri();
+    final URI[] testUris = new URI[] {
+        new URI(uri1.getScheme(), uri1.getAuthority(), "/", null, null),
+        new URI(uri2.getScheme(), uri2.getAuthority(), "/", null, null)
+    };
+
+    final Configuration testConf = new Configuration(conf);
+    testConf.setInt(IPC_CLIENT_CONNECT_MAX_RETRIES_KEY, 1);
+
+    final String testString = "Hello Nfly!";
+    final Path nflyRoot = new Path("/nflyroot");
+
+    ConfigUtil.addLinkNfly(testConf,
+        Constants.CONFIG_VIEWFS_DEFAULT_MOUNT_TABLE,
+        nflyRoot.toString(),
+        "minReplication=2," + repairKey + "=true", testUris);
+
+    final FileSystem nfly = FileSystem.get(URI.create("viewfs:///"), testConf);
+    // wd = /nflyroot/user/<user>
+    nfly.setWorkingDirectory(new Path(nflyRoot
+        + nfly.getWorkingDirectory().toUri().getPath()));
+
+    // 1. test mkdirs
+    final Path testDir = new Path("testdir1/sub1/sub3");
+    final Path testDir_tmp = new Path("testdir1/sub1/sub3_temp");
+    assertTrue(testDir + ": Failed to create!", nfly.mkdirs(testDir));
+
+    // Test renames
+    assertTrue(nfly.rename(testDir, testDir_tmp));
+    assertTrue(nfly.rename(testDir_tmp, testDir));
+
+    for (final URI testUri : testUris) {
+      final FileSystem fs = FileSystem.get(testUri, testConf);
+      assertTrue(testDir + " should exist!", fs.exists(testDir));
+    }
+
+    // 2. test write
+    final Path testFile = new Path("test.txt");
+    final FSDataOutputStream fsDos = nfly.create(testFile);
+    try {
+      fsDos.writeUTF(testString);
+    } finally {
+      fsDos.close();
+    }
+
+    for (final URI testUri : testUris) {
+      final FileSystem fs = FileSystem.get(testUri, testConf);
+      final FSDataInputStream fsdis = fs.open(testFile);
+      try {
+        assertEquals("Wrong file content", testString, fsdis.readUTF());
+      } finally {
+        fsdis.close();
+      }
+    }
+
+    // 3. test reads when one unavailable
+    //
+    // bring one NN down and read through nfly should still work
+    //
+    for (int i = 0; i < cluster.getNumNameNodes(); i++) {
+      cluster.shutdownNameNode(i);
+      FSDataInputStream fsDis = null;
+      try {
+        fsDis = nfly.open(testFile);
+        assertEquals("Wrong file content", testString, fsDis.readUTF());
+      } finally {
+        IOUtils.cleanupWithLogger(LOG, fsDis);
+        cluster.restartNameNode(i);
+      }
+    }
+
+    // both nodes are up again, test repair
+    final FileSystem fs1 = FileSystem.get(testUris[0], conf);
+    assertTrue(fs1.delete(testFile, false));
+    assertFalse(fs1.exists(testFile));
+    FSDataInputStream fsDis = null;
+    try {
+      fsDis = nfly.open(testFile);
+      assertEquals("Wrong file content", testString, fsDis.readUTF());
+      assertTrue(fs1.exists(testFile));
+    } finally {
+      IOUtils.cleanupWithLogger(LOG, fsDis);
+    }
+
+    // test most recent repair
+    if (repairKey == NflyFSystem.NflyKey.readMostRecent) {
+      final FileSystem fs2 = FileSystem.get(testUris[0], conf);
+      final long expectedMtime = fs2.getFileStatus(testFile)
+          .getModificationTime();
+
+      for (final URI testUri : testUris) {
+        final FileSystem fs = FileSystem.get(testUri, conf);
+        fs.setTimes(testFile, 1L, 1L);
+        assertEquals(testUri + "Set mtime failed!", 1L,
+            fs.getFileStatus(testFile).getModificationTime());
+        assertEquals("nfly file status wrong", expectedMtime,
+            nfly.getFileStatus(testFile).getModificationTime());
+        FSDataInputStream fsDis2 = null;
+        try {
+          fsDis2 = nfly.open(testFile);
+          assertEquals("Wrong file content", testString, fsDis2.readUTF());
+          // repair is done, now trying via normal fs
+          //
+          assertEquals("Repair most recent failed!", expectedMtime,
+              fs.getFileStatus(testFile).getModificationTime());
+        } finally {
+          IOUtils.cleanupWithLogger(LOG, fsDis2);
+        }
+      }
+    }
+  }
 }


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[03/37] hadoop git commit: Plan/ResourceAllocation data structure enhancements required to support recurring reservations in ReservationSystem.

Posted by ae...@apache.org.
Plan/ResourceAllocation data structure enhancements required to support recurring reservations in ReservationSystem.


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

Branch: refs/heads/HDFS-7240
Commit: 7996eca7dcfaa1bdf970e32022274f2699bef8a1
Parents: c5281a8
Author: Subru Krishnan <su...@apache.org>
Authored: Fri Sep 1 15:16:40 2017 -0700
Committer: Subru Krishnan <su...@apache.org>
Committed: Fri Sep 1 15:16:40 2017 -0700

----------------------------------------------------------------------
 .../hadoop/yarn/conf/YarnConfiguration.java     |   6 +
 .../yarn/conf/TestYarnConfigurationFields.java  |   4 +-
 .../reservation/AbstractReservationSystem.java  |  90 ++--
 .../AbstractSchedulerPlanFollower.java          | 183 ++++----
 .../reservation/InMemoryPlan.java               | 400 ++++++++++++-----
 .../InMemoryReservationAllocation.java          |  36 +-
 .../reservation/NoOverCommitPolicy.java         |   2 +-
 .../PeriodicRLESparseResourceAllocation.java    | 130 ++++--
 .../resourcemanager/reservation/PlanEdit.java   |  24 +-
 .../resourcemanager/reservation/PlanView.java   |  94 ++--
 .../RLESparseResourceAllocation.java            | 115 ++---
 .../reservation/ReservationAllocation.java      |  60 ++-
 .../reservation/ReservationInputValidator.java  | 134 +++---
 .../reservation/ReservationSystem.java          |   6 +-
 .../reservation/SharingPolicy.java              |  13 +-
 .../reservation/planning/Planner.java           |   2 +-
 .../reservation/planning/PlanningAlgorithm.java |   2 +-
 .../reservation/planning/StageAllocator.java    |   2 +-
 .../planning/StageAllocatorGreedy.java          |   2 +-
 .../planning/StageAllocatorGreedyRLE.java       |   5 +-
 .../planning/StageAllocatorLowCostAligned.java  |   4 +-
 .../reservation/ReservationSystemTestUtil.java  | 135 +++---
 .../reservation/TestInMemoryPlan.java           | 431 ++++++++++++-------
 ...TestPeriodicRLESparseResourceAllocation.java | 109 +++--
 .../TestRLESparseResourceAllocation.java        | 122 +++---
 .../planning/TestSimpleCapacityReplanner.java   |   8 +-
 26 files changed, 1342 insertions(+), 777 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/7996eca7/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
index 4944821..27ca957 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
@@ -262,6 +262,12 @@ public class YarnConfiguration extends Configuration {
   public static final long DEFAULT_RM_RESERVATION_SYSTEM_PLAN_FOLLOWER_TIME_STEP =
       1000L;
 
+  /** The maximum periodicity for the Reservation System. */
+  public static final String RM_RESERVATION_SYSTEM_MAX_PERIODICITY =
+      RM_PREFIX + "reservation-system.max-periodicity";
+  public static final long DEFAULT_RM_RESERVATION_SYSTEM_MAX_PERIODICITY =
+      86400000L;
+
   /**
    * Enable periodic monitor threads.
    * @see #RM_SCHEDULER_MONITOR_POLICIES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7996eca7/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/test/java/org/apache/hadoop/yarn/conf/TestYarnConfigurationFields.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/test/java/org/apache/hadoop/yarn/conf/TestYarnConfigurationFields.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/test/java/org/apache/hadoop/yarn/conf/TestYarnConfigurationFields.java
index bd7bf93..1d3111c 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/test/java/org/apache/hadoop/yarn/conf/TestYarnConfigurationFields.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/test/java/org/apache/hadoop/yarn/conf/TestYarnConfigurationFields.java
@@ -33,7 +33,7 @@ import org.apache.hadoop.conf.TestConfigurationFieldsBase;
  */
 public class TestYarnConfigurationFields extends TestConfigurationFieldsBase {
 
-  @SuppressWarnings("deprecation")
+  @SuppressWarnings({"deprecation", "methodlength"})
   @Override
   public void initializeMemberVariables() {
     xmlFilename = new String("yarn-default.xml");
@@ -69,6 +69,8 @@ public class TestYarnConfigurationFields extends TestConfigurationFieldsBase {
     configurationPropsToSkipCompare.add(YarnConfiguration
         .YARN_SECURITY_SERVICE_AUTHORIZATION_COLLECTOR_NODEMANAGER_PROTOCOL);
     configurationPropsToSkipCompare.add(YarnConfiguration.CURATOR_LEADER_ELECTOR);
+    configurationPropsToSkipCompare
+        .add(YarnConfiguration.RM_RESERVATION_SYSTEM_MAX_PERIODICITY);
 
     // Federation default configs to be ignored
     configurationPropsToSkipCompare

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7996eca7/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/AbstractReservationSystem.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/reservation/AbstractReservationSystem.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/AbstractReservationSystem.java
index 5ef4912..5b8772c 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/AbstractReservationSystem.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/AbstractReservationSystem.java
@@ -18,6 +18,17 @@
 
 package org.apache.hadoop.yarn.server.resourcemanager.reservation;
 
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Set;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+
 import org.apache.hadoop.classification.InterfaceAudience.LimitedPrivate;
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.conf.Configuration;
@@ -46,17 +57,6 @@ import org.apache.hadoop.yarn.util.resource.ResourceCalculator;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import java.util.HashMap;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.Set;
-import java.util.concurrent.ScheduledExecutorService;
-import java.util.concurrent.ScheduledThreadPoolExecutor;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicLong;
-import java.util.concurrent.locks.Lock;
-import java.util.concurrent.locks.ReentrantReadWriteLock;
-
 /**
  * This is the implementation of {@link ReservationSystem} based on the
  * {@link ResourceScheduler}
@@ -66,8 +66,8 @@ import java.util.concurrent.locks.ReentrantReadWriteLock;
 public abstract class AbstractReservationSystem extends AbstractService
     implements ReservationSystem {
 
-  private static final Logger LOG = LoggerFactory
-      .getLogger(AbstractReservationSystem.class);
+  private static final Logger LOG =
+      LoggerFactory.getLogger(AbstractReservationSystem.class);
 
   // private static final String DEFAULT_CAPACITY_SCHEDULER_PLAN
 
@@ -103,6 +103,8 @@ public abstract class AbstractReservationSystem extends AbstractService
 
   private boolean isRecoveryEnabled = false;
 
+  private long maxPeriodicity;
+
   /**
    * Construct the service.
    * 
@@ -143,36 +145,41 @@ public abstract class AbstractReservationSystem extends AbstractService
     this.conf = conf;
     scheduler = rmContext.getScheduler();
     // Get the plan step size
-    planStepSize =
-        conf.getTimeDuration(
-            YarnConfiguration.RM_RESERVATION_SYSTEM_PLAN_FOLLOWER_TIME_STEP,
-            YarnConfiguration.DEFAULT_RM_RESERVATION_SYSTEM_PLAN_FOLLOWER_TIME_STEP,
-            TimeUnit.MILLISECONDS);
+    planStepSize = conf.getTimeDuration(
+        YarnConfiguration.RM_RESERVATION_SYSTEM_PLAN_FOLLOWER_TIME_STEP,
+        YarnConfiguration.DEFAULT_RM_RESERVATION_SYSTEM_PLAN_FOLLOWER_TIME_STEP,
+        TimeUnit.MILLISECONDS);
     if (planStepSize < 0) {
       planStepSize =
           YarnConfiguration.DEFAULT_RM_RESERVATION_SYSTEM_PLAN_FOLLOWER_TIME_STEP;
     }
+    maxPeriodicity =
+        conf.getLong(YarnConfiguration.RM_RESERVATION_SYSTEM_MAX_PERIODICITY,
+            YarnConfiguration.DEFAULT_RM_RESERVATION_SYSTEM_MAX_PERIODICITY);
+    if (maxPeriodicity <= 0) {
+      maxPeriodicity =
+          YarnConfiguration.DEFAULT_RM_RESERVATION_SYSTEM_MAX_PERIODICITY;
+    }
     // Create a plan corresponding to every reservable queue
     Set<String> planQueueNames = scheduler.getPlanQueues();
     for (String planQueueName : planQueueNames) {
       Plan plan = initializePlan(planQueueName);
       plans.put(planQueueName, plan);
     }
-    isRecoveryEnabled = conf.getBoolean(
-        YarnConfiguration.RECOVERY_ENABLED,
+    isRecoveryEnabled = conf.getBoolean(YarnConfiguration.RECOVERY_ENABLED,
         YarnConfiguration.DEFAULT_RM_RECOVERY_ENABLED);
 
     if (conf.getBoolean(YarnConfiguration.YARN_RESERVATION_ACL_ENABLE,
-            YarnConfiguration.DEFAULT_YARN_RESERVATION_ACL_ENABLE) &&
-                    conf.getBoolean(YarnConfiguration.YARN_ACL_ENABLE,
-                            YarnConfiguration.DEFAULT_YARN_ACL_ENABLE)) {
+        YarnConfiguration.DEFAULT_YARN_RESERVATION_ACL_ENABLE)
+        && conf.getBoolean(YarnConfiguration.YARN_ACL_ENABLE,
+            YarnConfiguration.DEFAULT_YARN_ACL_ENABLE)) {
       reservationsACLsManager = new ReservationsACLsManager(scheduler, conf);
     }
   }
 
   private void loadPlan(String planName,
       Map<ReservationId, ReservationAllocationStateProto> reservations)
-          throws PlanningException {
+      throws PlanningException {
     Plan plan = plans.get(planName);
     Resource minAllocation = getMinAllocation();
     ResourceCalculator rescCalculator = getResourceCalculator();
@@ -248,8 +255,8 @@ public abstract class AbstractReservationSystem extends AbstractService
       Class<?> planFollowerPolicyClazz =
           conf.getClassByName(planFollowerPolicyClassName);
       if (PlanFollower.class.isAssignableFrom(planFollowerPolicyClazz)) {
-        return (PlanFollower) ReflectionUtils.newInstance(
-            planFollowerPolicyClazz, conf);
+        return (PlanFollower) ReflectionUtils
+            .newInstance(planFollowerPolicyClazz, conf);
       } else {
         throw new YarnRuntimeException("Class: " + planFollowerPolicyClassName
             + " not instance of " + PlanFollower.class.getCanonicalName());
@@ -257,7 +264,8 @@ public abstract class AbstractReservationSystem extends AbstractService
     } catch (ClassNotFoundException e) {
       throw new YarnRuntimeException(
           "Could not instantiate PlanFollowerPolicy: "
-              + planFollowerPolicyClassName, e);
+              + planFollowerPolicyClassName,
+          e);
     }
   }
 
@@ -371,9 +379,8 @@ public abstract class AbstractReservationSystem extends AbstractService
   public ReservationId getNewReservationId() {
     writeLock.lock();
     try {
-      ReservationId resId =
-          ReservationId.newInstance(ResourceManager.getClusterTimeStamp(),
-              resCounter.incrementAndGet());
+      ReservationId resId = ReservationId.newInstance(
+          ResourceManager.getClusterTimeStamp(), resCounter.incrementAndGet());
       LOG.info("Allocated new reservationId: " + resId);
       return resId;
     } finally {
@@ -390,8 +397,11 @@ public abstract class AbstractReservationSystem extends AbstractService
    * Get the default reservation system corresponding to the scheduler
    * 
    * @param scheduler the scheduler for which the reservation system is required
+   *
+   * @return the {@link ReservationSystem} based on the configured scheduler
    */
-  public static String getDefaultReservationSystem(ResourceScheduler scheduler) {
+  public static String getDefaultReservationSystem(
+      ResourceScheduler scheduler) {
     if (scheduler instanceof CapacityScheduler) {
       return CapacityReservationSystem.class.getName();
     } else if (scheduler instanceof FairScheduler) {
@@ -409,12 +419,11 @@ public abstract class AbstractReservationSystem extends AbstractService
     Resource maxAllocation = getMaxAllocation();
     ResourceCalculator rescCalc = getResourceCalculator();
     Resource totCap = getPlanQueueCapacity(planQueueName);
-    Plan plan =
-        new InMemoryPlan(getRootQueueMetrics(), adPolicy,
-            getAgent(planQueuePath), totCap, planStepSize, rescCalc,
-            minAllocation, maxAllocation, planQueueName,
-            getReplanner(planQueuePath), getReservationSchedulerConfiguration()
-            .getMoveOnExpiry(planQueuePath), rmContext);
+    Plan plan = new InMemoryPlan(getRootQueueMetrics(), adPolicy,
+        getAgent(planQueuePath), totCap, planStepSize, rescCalc, minAllocation,
+        maxAllocation, planQueueName, getReplanner(planQueuePath),
+        getReservationSchedulerConfiguration().getMoveOnExpiry(planQueuePath),
+        maxPeriodicity, rmContext);
     LOG.info("Initialized plan {} based on reservable queue {}",
         plan.toString(), planQueueName);
     return plan;
@@ -477,8 +486,8 @@ public abstract class AbstractReservationSystem extends AbstractService
       Class<?> admissionPolicyClazz =
           conf.getClassByName(admissionPolicyClassName);
       if (SharingPolicy.class.isAssignableFrom(admissionPolicyClazz)) {
-        return (SharingPolicy) ReflectionUtils.newInstance(
-            admissionPolicyClazz, conf);
+        return (SharingPolicy) ReflectionUtils.newInstance(admissionPolicyClazz,
+            conf);
       } else {
         throw new YarnRuntimeException("Class: " + admissionPolicyClassName
             + " not instance of " + SharingPolicy.class.getCanonicalName());
@@ -493,8 +502,7 @@ public abstract class AbstractReservationSystem extends AbstractService
     return this.reservationsACLsManager;
   }
 
-  protected abstract ReservationSchedulerConfiguration
-      getReservationSchedulerConfiguration();
+  protected abstract ReservationSchedulerConfiguration getReservationSchedulerConfiguration();
 
   protected abstract String getPlanQueuePath(String planQueueName);
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7996eca7/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/AbstractSchedulerPlanFollower.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/reservation/AbstractSchedulerPlanFollower.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/AbstractSchedulerPlanFollower.java
index 90357e3..9b6a0b0 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/AbstractSchedulerPlanFollower.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/AbstractSchedulerPlanFollower.java
@@ -18,6 +18,14 @@
 
 package org.apache.hadoop.yarn.server.resourcemanager.reservation;
 
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ReservationId;
 import org.apache.hadoop.yarn.api.records.Resource;
@@ -33,24 +41,17 @@ import org.apache.hadoop.yarn.util.resource.Resources;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.Comparator;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Set;
-
 public abstract class AbstractSchedulerPlanFollower implements PlanFollower {
-  private static final Logger LOG = LoggerFactory
-      .getLogger(AbstractSchedulerPlanFollower.class);
+  private static final Logger LOG =
+      LoggerFactory.getLogger(AbstractSchedulerPlanFollower.class);
 
   protected Collection<Plan> plans = new ArrayList<Plan>();
   protected YarnScheduler scheduler;
   protected Clock clock;
 
   @Override
-  public void init(Clock clock, ResourceScheduler sched, Collection<Plan> plans) {
+  public void init(Clock clock, ResourceScheduler sched,
+      Collection<Plan> plans) {
     this.clock = clock;
     this.scheduler = sched;
     this.plans.addAll(plans);
@@ -71,7 +72,7 @@ public abstract class AbstractSchedulerPlanFollower implements PlanFollower {
 
   @Override
   public synchronized void synchronizePlan(Plan plan, boolean shouldReplan) {
-     String planQueueName = plan.getQueueName();
+    String planQueueName = plan.getQueueName();
     if (LOG.isDebugEnabled()) {
       LOG.debug("Running plan follower edit policy for plan: " + planQueueName);
     }
@@ -82,12 +83,14 @@ public abstract class AbstractSchedulerPlanFollower implements PlanFollower {
       now += step - (now % step);
     }
     Queue planQueue = getPlanQueue(planQueueName);
-    if (planQueue == null) return;
+    if (planQueue == null) {
+      return;
+    }
 
     // first we publish to the plan the current availability of resources
     Resource clusterResources = scheduler.getClusterResource();
-    Resource planResources = getPlanResources(plan, planQueue,
-        clusterResources);
+    Resource planResources =
+        getPlanResources(plan, planQueue, clusterResources);
     Set<ReservationAllocation> currentReservations =
         plan.getReservationsAtTime(now);
     Set<String> curReservationNames = new HashSet<String>();
@@ -95,12 +98,11 @@ public abstract class AbstractSchedulerPlanFollower implements PlanFollower {
     int numRes = getReservedResources(now, currentReservations,
         curReservationNames, reservedResources);
     // create the default reservation queue if it doesnt exist
-    String defReservationId = getReservationIdFromQueueName(planQueueName) +
-        ReservationConstants.DEFAULT_QUEUE_SUFFIX;
-    String defReservationQueue = getReservationQueueName(planQueueName,
-        defReservationId);
-    createDefaultReservationQueue(planQueueName, planQueue,
-        defReservationId);
+    String defReservationId = getReservationIdFromQueueName(planQueueName)
+        + ReservationConstants.DEFAULT_QUEUE_SUFFIX;
+    String defReservationQueue =
+        getReservationQueueName(planQueueName, defReservationId);
+    createDefaultReservationQueue(planQueueName, planQueue, defReservationId);
     curReservationNames.add(defReservationId);
     // if the resources dedicated to this plan has shrunk invoke replanner
     boolean shouldResize = false;
@@ -149,10 +151,8 @@ public abstract class AbstractSchedulerPlanFollower implements PlanFollower {
       // sort allocations from the one giving up the most resources, to the
       // one asking for the most avoid order-of-operation errors that
       // temporarily violate 100% capacity bound
-      List<ReservationAllocation> sortedAllocations =
-          sortByDelta(
-              new ArrayList<ReservationAllocation>(currentReservations), now,
-              plan);
+      List<ReservationAllocation> sortedAllocations = sortByDelta(
+          new ArrayList<ReservationAllocation>(currentReservations), now, plan);
       for (ReservationAllocation res : sortedAllocations) {
         String currResId = res.getReservationId().toString();
         if (curReservationNames.contains(currResId)) {
@@ -163,10 +163,9 @@ public abstract class AbstractSchedulerPlanFollower implements PlanFollower {
         if (planResources.getMemorySize() > 0
             && planResources.getVirtualCores() > 0) {
           if (shouldResize) {
-            capToAssign =
-                calculateReservationToPlanProportion(
-                    plan.getResourceCalculator(), planResources,
-                    reservedResources, capToAssign);
+            capToAssign = calculateReservationToPlanProportion(
+                plan.getResourceCalculator(), planResources, reservedResources,
+                capToAssign);
           }
           targetCapacity =
               calculateReservationToPlanRatio(plan.getResourceCalculator(),
@@ -185,7 +184,8 @@ public abstract class AbstractSchedulerPlanFollower implements PlanFollower {
           maxCapacity = targetCapacity;
         }
         try {
-          setQueueEntitlement(planQueueName, currResId, targetCapacity, maxCapacity);
+          setQueueEntitlement(planQueueName, currResId, targetCapacity,
+              maxCapacity);
         } catch (YarnException e) {
           LOG.warn("Exception while trying to size reservation for plan: {}",
               currResId, planQueueName, e);
@@ -196,9 +196,10 @@ public abstract class AbstractSchedulerPlanFollower implements PlanFollower {
     // compute the default queue capacity
     float defQCap = 1.0f - totalAssignedCapacity;
     if (LOG.isDebugEnabled()) {
-      LOG.debug("PlanFollowerEditPolicyTask: total Plan Capacity: {} "
-          + "currReservation: {} default-queue capacity: {}", planResources,
-          numRes, defQCap);
+      LOG.debug(
+          "PlanFollowerEditPolicyTask: total Plan Capacity: {} "
+              + "currReservation: {} default-queue capacity: {}",
+          planResources, numRes, defQCap);
     }
     // set the default queue to eat-up all remaining capacity
     try {
@@ -225,12 +226,11 @@ public abstract class AbstractSchedulerPlanFollower implements PlanFollower {
   }
 
   protected void setQueueEntitlement(String planQueueName, String currResId,
-      float targetCapacity,
-      float maxCapacity) throws YarnException {
-    String reservationQueueName = getReservationQueueName(planQueueName,
-        currResId);
-    scheduler.setEntitlement(reservationQueueName, new QueueEntitlement(
-        targetCapacity, maxCapacity));
+      float targetCapacity, float maxCapacity) throws YarnException {
+    String reservationQueueName =
+        getReservationQueueName(planQueueName, currResId);
+    scheduler.setEntitlement(reservationQueueName,
+        new QueueEntitlement(targetCapacity, maxCapacity));
   }
 
   // Schedulers have different ways of naming queues. See YARN-2773
@@ -244,14 +244,21 @@ public abstract class AbstractSchedulerPlanFollower implements PlanFollower {
    * Then move all apps in the set of queues to the parent plan queue's default
    * reservation queue if move is enabled. Finally cleanups the queue by killing
    * any apps (if move is disabled or move failed) and removing the queue
+   *
+   * @param planQueueName the name of {@code PlanQueue}
+   * @param shouldMove flag to indicate if any running apps should be moved or
+   *          killed
+   * @param toRemove the remnant apps to clean up
+   * @param defReservationQueue the default {@code ReservationQueue} of the
+   *          {@link Plan}
    */
-  protected void cleanupExpiredQueues(String planQueueName,
-      boolean shouldMove, Set<String> toRemove, String defReservationQueue) {
+  protected void cleanupExpiredQueues(String planQueueName, boolean shouldMove,
+      Set<String> toRemove, String defReservationQueue) {
     for (String expiredReservationId : toRemove) {
       try {
         // reduce entitlement to 0
-        String expiredReservation = getReservationQueueName(planQueueName,
-            expiredReservationId);
+        String expiredReservation =
+            getReservationQueueName(planQueueName, expiredReservationId);
         setQueueEntitlement(planQueueName, expiredReservation, 0.0f, 0.0f);
         if (shouldMove) {
           moveAppsInQueueSync(expiredReservation, defReservationQueue);
@@ -275,7 +282,7 @@ public abstract class AbstractSchedulerPlanFollower implements PlanFollower {
    * reservation queue in a synchronous fashion
    */
   private void moveAppsInQueueSync(String expiredReservation,
-                                   String defReservationQueue) {
+      String defReservationQueue) {
     List<ApplicationAttemptId> activeApps =
         scheduler.getAppsInQueue(expiredReservation);
     if (activeApps.isEmpty()) {
@@ -287,16 +294,16 @@ public abstract class AbstractSchedulerPlanFollower implements PlanFollower {
         scheduler.moveApplication(app.getApplicationId(), defReservationQueue);
       } catch (YarnException e) {
         LOG.warn(
-            "Encountered unexpected error during migration of application: {}" +
-                " from reservation: {}",
+            "Encountered unexpected error during migration of application: {}"
+                + " from reservation: {}",
             app, expiredReservation, e);
       }
     }
   }
 
-  protected int getReservedResources(long now, Set<ReservationAllocation>
-      currentReservations, Set<String> curReservationNames,
-                                     Resource reservedResources) {
+  protected int getReservedResources(long now,
+      Set<ReservationAllocation> currentReservations,
+      Set<String> curReservationNames, Resource reservedResources) {
     int numRes = 0;
     if (currentReservations != null) {
       numRes = currentReservations.size();
@@ -312,23 +319,30 @@ public abstract class AbstractSchedulerPlanFollower implements PlanFollower {
    * Sort in the order from the least new amount of resources asked (likely
    * negative) to the highest. This prevents "order-of-operation" errors related
    * to exceeding 100% capacity temporarily.
+   *
+   * @param currentReservations the currently active reservations
+   * @param now the current time
+   * @param plan the {@link Plan} that is being considered
+   *
+   * @return the sorted list of {@link ReservationAllocation}s
    */
   protected List<ReservationAllocation> sortByDelta(
       List<ReservationAllocation> currentReservations, long now, Plan plan) {
-    Collections.sort(currentReservations, new ReservationAllocationComparator(
-        now, this, plan));
+    Collections.sort(currentReservations,
+        new ReservationAllocationComparator(now, this, plan));
     return currentReservations;
   }
 
   /**
-   * Get queue associated with reservable queue named
-   * @param planQueueName Name of the reservable queue
+   * Get queue associated with reservable queue named.
+   *
+   * @param planQueueName name of the reservable queue
    * @return queue associated with the reservable queue
    */
   protected abstract Queue getPlanQueue(String planQueueName);
 
   /**
-   * Resizes reservations based on currently available resources
+   * Resizes reservations based on currently available resources.
    */
   private Resource calculateReservationToPlanProportion(
       ResourceCalculator rescCalculator, Resource availablePlanResources,
@@ -338,7 +352,7 @@ public abstract class AbstractSchedulerPlanFollower implements PlanFollower {
   }
 
   /**
-   * Calculates ratio of reservationResources to planResources
+   * Calculates ratio of reservationResources to planResources.
    */
   private float calculateReservationToPlanRatio(
       ResourceCalculator rescCalculator, Resource clusterResources,
@@ -348,7 +362,7 @@ public abstract class AbstractSchedulerPlanFollower implements PlanFollower {
   }
 
   /**
-   * Check if plan resources are less than expected reservation resources
+   * Check if plan resources are less than expected reservation resources.
    */
   private boolean arePlanResourcesLessThanReservations(
       ResourceCalculator rescCalculator, Resource clusterResources,
@@ -358,38 +372,56 @@ public abstract class AbstractSchedulerPlanFollower implements PlanFollower {
   }
 
   /**
-   * Get a list of reservation queues for this planQueue
+   * Get a list of reservation queues for this planQueue.
+   *
+   * @param planQueue the queue for the current {@link Plan}
+   *
+   * @return the queues corresponding to the reservations
    */
   protected abstract List<? extends Queue> getChildReservationQueues(
       Queue planQueue);
 
   /**
-   * Add a new reservation queue for reservation currResId for this planQueue
+   * Add a new reservation queue for reservation currResId for this planQueue.
    */
-  protected abstract void addReservationQueue(
-      String planQueueName, Queue queue, String currResId);
+  protected abstract void addReservationQueue(String planQueueName, Queue queue,
+      String currResId);
 
   /**
-   * Creates the default reservation queue for use when no reservation is
-   * used for applications submitted to this planQueue
+   * Creates the default reservation queue for use when no reservation is used
+   * for applications submitted to this planQueue.
+   *
+   * @param planQueueName name of the reservable queue
+   * @param queue the queue for the current {@link Plan}
+   * @param defReservationQueue name of the default {@code ReservationQueue}
    */
-  protected abstract void createDefaultReservationQueue(
-      String planQueueName, Queue queue, String defReservationQueue);
+  protected abstract void createDefaultReservationQueue(String planQueueName,
+      Queue queue, String defReservationQueue);
 
   /**
-   * Get plan resources for this planQueue
+   * Get plan resources for this planQueue.
+   *
+   * @param plan the current {@link Plan} being considered
+   * @param clusterResources the resources available in the cluster
+   *
+   * @return the resources allocated to the specified {@link Plan}
    */
-  protected abstract Resource getPlanResources(
-      Plan plan, Queue queue, Resource clusterResources);
+  protected abstract Resource getPlanResources(Plan plan, Queue queue,
+      Resource clusterResources);
 
   /**
    * Get reservation queue resources if it exists otherwise return null.
+   *
+   * @param plan the current {@link Plan} being considered
+   * @param reservationId the identifier of the reservation
+   *
+   * @return the resources allocated to the specified reservation
    */
   protected abstract Resource getReservationQueueResourceIfExists(Plan plan,
       ReservationId reservationId);
 
-  private static class ReservationAllocationComparator implements
-      Comparator<ReservationAllocation> {
+  private static class ReservationAllocationComparator
+      implements Comparator<ReservationAllocation> {
     AbstractSchedulerPlanFollower planFollower;
     long now;
     Plan plan;
@@ -404,14 +436,12 @@ public abstract class AbstractSchedulerPlanFollower implements PlanFollower {
     private Resource getUnallocatedReservedResources(
         ReservationAllocation reservation) {
       Resource resResource;
-      Resource reservationResource = planFollower
-          .getReservationQueueResourceIfExists
-              (plan, reservation.getReservationId());
+      Resource reservationResource =
+          planFollower.getReservationQueueResourceIfExists(plan,
+              reservation.getReservationId());
       if (reservationResource != null) {
-        resResource =
-            Resources.subtract(
-                reservation.getResourcesAtTime(now),
-                reservationResource);
+        resResource = Resources.subtract(reservation.getResourcesAtTime(now),
+            reservationResource);
       } else {
         resResource = reservation.getResourcesAtTime(now);
       }
@@ -428,4 +458,3 @@ public abstract class AbstractSchedulerPlanFollower implements PlanFollower {
     }
   }
 }
-

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7996eca7/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/InMemoryPlan.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/reservation/InMemoryPlan.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/InMemoryPlan.java
index 783fd09..9eb1820 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/InMemoryPlan.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/InMemoryPlan.java
@@ -6,9 +6,9 @@
  * 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
- *
+ * <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.
@@ -33,9 +33,10 @@ import java.util.concurrent.locks.ReentrantReadWriteLock;
 
 import org.apache.hadoop.yarn.api.records.ReservationId;
 import org.apache.hadoop.yarn.api.records.Resource;
-import org.apache.hadoop.yarn.server.resourcemanager.reservation.RLESparseResourceAllocation.RLEOperator;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
 import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore;
+import org.apache.hadoop.yarn.server.resourcemanager.reservation.RLESparseResourceAllocation.RLEOperator;
 import org.apache.hadoop.yarn.server.resourcemanager.reservation.exceptions.PlanningException;
 import org.apache.hadoop.yarn.server.resourcemanager.reservation.planning.Planner;
 import org.apache.hadoop.yarn.server.resourcemanager.reservation.planning.ReservationAgent;
@@ -64,9 +65,14 @@ public class InMemoryPlan implements Plan {
 
   private RLESparseResourceAllocation rleSparseVector;
 
+  private PeriodicRLESparseResourceAllocation periodicRle;
+
   private Map<String, RLESparseResourceAllocation> userResourceAlloc =
       new HashMap<String, RLESparseResourceAllocation>();
 
+  private Map<String, RLESparseResourceAllocation> userPeriodicResourceAlloc =
+      new HashMap<String, RLESparseResourceAllocation>();
+
   private Map<String, RLESparseResourceAllocation> userActiveReservationCount =
       new HashMap<String, RLESparseResourceAllocation>();
 
@@ -96,15 +102,27 @@ public class InMemoryPlan implements Plan {
       String queueName, Planner replanner, boolean getMoveOnExpiry,
       RMContext rmContext) {
     this(queueMetrics, policy, agent, totalCapacity, step, resCalc, minAlloc,
-        maxAlloc, queueName, replanner, getMoveOnExpiry, rmContext,
-        new UTCClock());
+        maxAlloc, queueName, replanner, getMoveOnExpiry,
+        YarnConfiguration.DEFAULT_RM_RESERVATION_SYSTEM_MAX_PERIODICITY,
+        rmContext);
   }
 
   public InMemoryPlan(QueueMetrics queueMetrics, SharingPolicy policy,
       ReservationAgent agent, Resource totalCapacity, long step,
       ResourceCalculator resCalc, Resource minAlloc, Resource maxAlloc,
       String queueName, Planner replanner, boolean getMoveOnExpiry,
-      RMContext rmContext, Clock clock) {
+      long maxPeriodicty, RMContext rmContext) {
+    this(queueMetrics, policy, agent, totalCapacity, step, resCalc, minAlloc,
+        maxAlloc, queueName, replanner, getMoveOnExpiry, maxPeriodicty,
+        rmContext, new UTCClock());
+  }
+
+  @SuppressWarnings("checkstyle:parameternumber")
+  public InMemoryPlan(QueueMetrics queueMetrics, SharingPolicy policy,
+      ReservationAgent agent, Resource totalCapacity, long step,
+      ResourceCalculator resCalc, Resource minAlloc, Resource maxAlloc,
+      String queueName, Planner replanner, boolean getMoveOnExpiry,
+      long maxPeriodicty, RMContext rmContext, Clock clock) {
     this.queueMetrics = queueMetrics;
     this.policy = policy;
     this.agent = agent;
@@ -114,6 +132,8 @@ public class InMemoryPlan implements Plan {
     this.minAlloc = minAlloc;
     this.maxAlloc = maxAlloc;
     this.rleSparseVector = new RLESparseResourceAllocation(resCalc);
+    this.periodicRle =
+        new PeriodicRLESparseResourceAllocation(resCalc, maxPeriodicty);
     this.queueName = queueName;
     this.replanner = replanner;
     this.getMoveOnExpiry = getMoveOnExpiry;
@@ -126,6 +146,39 @@ public class InMemoryPlan implements Plan {
     return queueMetrics;
   }
 
+  private RLESparseResourceAllocation getUserRLEResourceAllocation(String user,
+      long period) {
+    RLESparseResourceAllocation resAlloc = null;
+    if (period > 0) {
+      if (userPeriodicResourceAlloc.containsKey(user)) {
+        resAlloc = userPeriodicResourceAlloc.get(user);
+      } else {
+        resAlloc = new PeriodicRLESparseResourceAllocation(resCalc,
+            periodicRle.getTimePeriod());
+        userPeriodicResourceAlloc.put(user, resAlloc);
+      }
+    } else {
+      if (userResourceAlloc.containsKey(user)) {
+        resAlloc = userResourceAlloc.get(user);
+      } else {
+        resAlloc = new RLESparseResourceAllocation(resCalc);
+        userResourceAlloc.put(user, resAlloc);
+      }
+    }
+    return resAlloc;
+  }
+
+  private void gcUserRLEResourceAllocation(String user, long period) {
+    if (period > 0) {
+      if (userPeriodicResourceAlloc.get(user).isEmpty()) {
+        userPeriodicResourceAlloc.remove(user);
+      }
+    } else {
+      if (userResourceAlloc.get(user).isEmpty()) {
+        userResourceAlloc.remove(user);
+      }
+    }
+  }
 
   private void incrementAllocation(ReservationAllocation reservation) {
     assert (readWriteLock.isWriteLockedByCurrentThread());
@@ -133,11 +186,10 @@ public class InMemoryPlan implements Plan {
         reservation.getAllocationRequests();
     // check if we have encountered the user earlier and if not add an entry
     String user = reservation.getUser();
-    RLESparseResourceAllocation resAlloc = userResourceAlloc.get(user);
-    if (resAlloc == null) {
-      resAlloc = new RLESparseResourceAllocation(resCalc);
-      userResourceAlloc.put(user, resAlloc);
-    }
+    long period = reservation.getPeriodicity();
+    RLESparseResourceAllocation resAlloc =
+        getUserRLEResourceAllocation(user, period);
+
     RLESparseResourceAllocation resCount = userActiveReservationCount.get(user);
     if (resCount == null) {
       resCount = new RLESparseResourceAllocation(resCalc);
@@ -149,14 +201,43 @@ public class InMemoryPlan implements Plan {
 
     for (Map.Entry<ReservationInterval, Resource> r : allocationRequests
         .entrySet()) {
-      resAlloc.addInterval(r.getKey(), r.getValue());
-      rleSparseVector.addInterval(r.getKey(), r.getValue());
-      if (Resources.greaterThan(resCalc, totalCapacity, r.getValue(),
-          ZERO_RESOURCE)) {
-        earliestActive = Math.min(earliestActive, r.getKey().getStartTime());
-        latestActive = Math.max(latestActive, r.getKey().getEndTime());
+
+      if (period > 0L) {
+        for (int i = 0; i < periodicRle.getTimePeriod() / period; i++) {
+
+          long rStart = r.getKey().getStartTime() + i * period;
+          long rEnd = r.getKey().getEndTime() + i * period;
+
+          // handle wrap-around
+          if (rEnd > periodicRle.getTimePeriod()) {
+            long diff = rEnd - periodicRle.getTimePeriod();
+            rEnd = periodicRle.getTimePeriod();
+            ReservationInterval newInterval = new ReservationInterval(0, diff);
+            periodicRle.addInterval(newInterval, r.getValue());
+            resAlloc.addInterval(newInterval, r.getValue());
+          }
+
+          ReservationInterval newInterval =
+              new ReservationInterval(rStart, rEnd);
+          periodicRle.addInterval(newInterval, r.getValue());
+          resAlloc.addInterval(newInterval, r.getValue());
+        }
+
+      } else {
+        rleSparseVector.addInterval(r.getKey(), r.getValue());
+        resAlloc.addInterval(r.getKey(), r.getValue());
+        if (Resources.greaterThan(resCalc, totalCapacity, r.getValue(),
+            ZERO_RESOURCE)) {
+          earliestActive = Math.min(earliestActive, r.getKey().getStartTime());
+          latestActive = Math.max(latestActive, r.getKey().getEndTime());
+        }
       }
     }
+    // periodic reservations are active from start time and good till cancelled
+    if (period > 0L) {
+      earliestActive = reservation.getStartTime();
+      latestActive = Long.MAX_VALUE;
+    }
     resCount.addInterval(new ReservationInterval(earliestActive, latestActive),
         Resource.newInstance(1, 1));
   }
@@ -166,27 +247,55 @@ public class InMemoryPlan implements Plan {
     Map<ReservationInterval, Resource> allocationRequests =
         reservation.getAllocationRequests();
     String user = reservation.getUser();
-    RLESparseResourceAllocation resAlloc = userResourceAlloc.get(user);
+    long period = reservation.getPeriodicity();
+    RLESparseResourceAllocation resAlloc =
+        getUserRLEResourceAllocation(user, period);
 
     long earliestActive = Long.MAX_VALUE;
     long latestActive = Long.MIN_VALUE;
     for (Map.Entry<ReservationInterval, Resource> r : allocationRequests
         .entrySet()) {
-      resAlloc.removeInterval(r.getKey(), r.getValue());
-      rleSparseVector.removeInterval(r.getKey(), r.getValue());
-      if (Resources.greaterThan(resCalc, totalCapacity, r.getValue(),
-          ZERO_RESOURCE)) {
-        earliestActive = Math.min(earliestActive, r.getKey().getStartTime());
-        latestActive = Math.max(latestActive, r.getKey().getEndTime());
+      if (period > 0L) {
+        for (int i = 0; i < periodicRle.getTimePeriod() / period; i++) {
+
+          long rStart = r.getKey().getStartTime() + i * period;
+          long rEnd = r.getKey().getEndTime() + i * period;
+
+          // handle wrap-around
+          if (rEnd > periodicRle.getTimePeriod()) {
+            long diff = rEnd - periodicRle.getTimePeriod();
+            rEnd = periodicRle.getTimePeriod();
+            ReservationInterval newInterval = new ReservationInterval(0, diff);
+            periodicRle.removeInterval(newInterval, r.getValue());
+            resAlloc.removeInterval(newInterval, r.getValue());
+          }
+
+          ReservationInterval newInterval =
+              new ReservationInterval(rStart, rEnd);
+          periodicRle.removeInterval(newInterval, r.getValue());
+          resAlloc.removeInterval(newInterval, r.getValue());
+        }
+      } else {
+        rleSparseVector.removeInterval(r.getKey(), r.getValue());
+        resAlloc.removeInterval(r.getKey(), r.getValue());
+        if (Resources.greaterThan(resCalc, totalCapacity, r.getValue(),
+            ZERO_RESOURCE)) {
+          earliestActive = Math.min(earliestActive, r.getKey().getStartTime());
+          latestActive = Math.max(latestActive, r.getKey().getEndTime());
+        }
       }
     }
-    if (resAlloc.isEmpty()) {
-      userResourceAlloc.remove(user);
-    }
+    gcUserRLEResourceAllocation(user, period);
 
     RLESparseResourceAllocation resCount = userActiveReservationCount.get(user);
-    resCount.removeInterval(new ReservationInterval(earliestActive,
-        latestActive), Resource.newInstance(1, 1));
+    // periodic reservations are active from start time and good till cancelled
+    if (period > 0L) {
+      earliestActive = reservation.getStartTime();
+      latestActive = Long.MAX_VALUE;
+    }
+    resCount.removeInterval(
+        new ReservationInterval(earliestActive, latestActive),
+        Resource.newInstance(1, 1));
     if (resCount.isEmpty()) {
       userActiveReservationCount.remove(user);
     }
@@ -198,9 +307,9 @@ public class InMemoryPlan implements Plan {
       if (currentReservations != null) {
         Set<ReservationAllocation> flattenedReservations =
             new TreeSet<ReservationAllocation>();
-        for (Set<InMemoryReservationAllocation> reservationEntries :
-            currentReservations.values()) {
-          flattenedReservations.addAll(reservationEntries);
+        for (Set<InMemoryReservationAllocation> res : currentReservations
+            .values()) {
+          flattenedReservations.addAll(res);
         }
         return flattenedReservations;
       } else {
@@ -218,19 +327,16 @@ public class InMemoryPlan implements Plan {
     InMemoryReservationAllocation inMemReservation =
         (InMemoryReservationAllocation) reservation;
     if (inMemReservation.getUser() == null) {
-      String errMsg =
-          "The specified Reservation with ID "
-              + inMemReservation.getReservationId()
-              + " is not mapped to any user";
+      String errMsg = "The specified Reservation with ID "
+          + inMemReservation.getReservationId() + " is not mapped to any user";
       LOG.error(errMsg);
       throw new IllegalArgumentException(errMsg);
     }
     writeLock.lock();
     try {
       if (reservationTable.containsKey(inMemReservation.getReservationId())) {
-        String errMsg =
-            "The specified Reservation with ID "
-                + inMemReservation.getReservationId() + " already exists";
+        String errMsg = "The specified Reservation with ID "
+            + inMemReservation.getReservationId() + " already exists";
         LOG.error(errMsg);
         throw new IllegalArgumentException(errMsg);
       }
@@ -246,9 +352,8 @@ public class InMemoryPlan implements Plan {
               getQueueName(), inMemReservation.getReservationId().toString());
         }
       }
-      ReservationInterval searchInterval =
-          new ReservationInterval(inMemReservation.getStartTime(),
-              inMemReservation.getEndTime());
+      ReservationInterval searchInterval = new ReservationInterval(
+          inMemReservation.getStartTime(), inMemReservation.getEndTime());
       Set<InMemoryReservationAllocation> reservations =
           currentReservations.get(searchInterval);
       if (reservations == null) {
@@ -280,9 +385,8 @@ public class InMemoryPlan implements Plan {
       ReservationId resId = reservation.getReservationId();
       ReservationAllocation currReservation = getReservationById(resId);
       if (currReservation == null) {
-        String errMsg =
-            "The specified Reservation with ID " + resId
-                + " does not exist in the plan";
+        String errMsg = "The specified Reservation with ID " + resId
+            + " does not exist in the plan";
         LOG.error(errMsg);
         throw new IllegalArgumentException(errMsg);
       }
@@ -318,9 +422,8 @@ public class InMemoryPlan implements Plan {
 
   private boolean removeReservation(ReservationAllocation reservation) {
     assert (readWriteLock.isWriteLockedByCurrentThread());
-    ReservationInterval searchInterval =
-        new ReservationInterval(reservation.getStartTime(),
-            reservation.getEndTime());
+    ReservationInterval searchInterval = new ReservationInterval(
+        reservation.getStartTime(), reservation.getEndTime());
     Set<InMemoryReservationAllocation> reservations =
         currentReservations.get(searchInterval);
     if (reservations != null) {
@@ -337,16 +440,15 @@ public class InMemoryPlan implements Plan {
         currentReservations.remove(searchInterval);
       }
     } else {
-      String errMsg =
-          "The specified Reservation with ID " + reservation.getReservationId()
-              + " does not exist in the plan";
+      String errMsg = "The specified Reservation with ID "
+          + reservation.getReservationId() + " does not exist in the plan";
       LOG.error(errMsg);
       throw new IllegalArgumentException(errMsg);
     }
     reservationTable.remove(reservation.getReservationId());
     decrementAllocation(reservation);
     LOG.info("Sucessfully deleted reservation: {} in plan.",
-            reservation.getReservationId());
+        reservation.getReservationId());
     return true;
   }
 
@@ -356,9 +458,8 @@ public class InMemoryPlan implements Plan {
     try {
       ReservationAllocation reservation = getReservationById(reservationID);
       if (reservation == null) {
-        String errMsg =
-            "The specified Reservation with ID " + reservationID
-                + " does not exist in the plan";
+        String errMsg = "The specified Reservation with ID " + reservationID
+            + " does not exist in the plan";
         LOG.error(errMsg);
         throw new IllegalArgumentException(errMsg);
       }
@@ -453,66 +554,90 @@ public class InMemoryPlan implements Plan {
       long start, long end) {
     readLock.lock();
     try {
+      // merge periodic and non-periodic allocations
       RLESparseResourceAllocation userResAlloc = userResourceAlloc.get(user);
+      RLESparseResourceAllocation userPeriodicResAlloc =
+          userPeriodicResourceAlloc.get(user);
 
+      if (userResAlloc != null && userPeriodicResAlloc != null) {
+        return RLESparseResourceAllocation.merge(resCalc, totalCapacity,
+            userResAlloc, userPeriodicResAlloc, RLEOperator.add, start, end);
+      }
       if (userResAlloc != null) {
         return userResAlloc.getRangeOverlapping(start, end);
-      } else {
-        return new RLESparseResourceAllocation(resCalc);
       }
+      if (userPeriodicResAlloc != null) {
+        return userPeriodicResAlloc.getRangeOverlapping(start, end);
+      }
+    } catch (PlanningException e) {
+      LOG.warn("Exception while trying to merge periodic"
+          + " and non-periodic user allocations: {}", e.getMessage(), e);
     } finally {
       readLock.unlock();
     }
+    return new RLESparseResourceAllocation(resCalc);
   }
 
   @Override
   public Resource getTotalCommittedResources(long t) {
     readLock.lock();
     try {
-      return rleSparseVector.getCapacityAtTime(t);
+      return Resources.add(rleSparseVector.getCapacityAtTime(t),
+          periodicRle.getCapacityAtTime(t));
     } finally {
       readLock.unlock();
     }
   }
 
   @Override
-  public Set<ReservationAllocation> getReservations(ReservationId
-                    reservationID, ReservationInterval interval) {
+  public Set<ReservationAllocation> getReservations(ReservationId reservationID,
+      ReservationInterval interval) {
     return getReservations(reservationID, interval, null);
   }
 
   @Override
-  public Set<ReservationAllocation> getReservations(ReservationId
-                    reservationID, ReservationInterval interval, String user) {
+  public Set<ReservationAllocation> getReservations(ReservationId reservationID,
+      ReservationInterval interval, String user) {
     if (reservationID != null) {
       ReservationAllocation allocation = getReservationById(reservationID);
-      if (allocation == null){
+      if (allocation == null) {
         return Collections.emptySet();
       }
       return Collections.singleton(allocation);
     }
 
-    long startTime = interval == null? 0 : interval.getStartTime();
-    long endTime = interval == null? Long.MAX_VALUE : interval.getEndTime();
+    long startTime = interval == null ? 0 : interval.getStartTime();
+    long endTime = interval == null ? Long.MAX_VALUE : interval.getEndTime();
 
     ReservationInterval searchInterval =
-            new ReservationInterval(endTime, Long.MAX_VALUE);
+        new ReservationInterval(endTime, Long.MAX_VALUE);
     readLock.lock();
     try {
-      SortedMap<ReservationInterval, Set<InMemoryReservationAllocation>>
-            reservations = currentReservations.headMap(searchInterval, true);
-      if (!reservations.isEmpty()) {
-        Set<ReservationAllocation> flattenedReservations =
-                new HashSet<>();
-        for (Set<InMemoryReservationAllocation> reservationEntries :
-                reservations.values()) {
-          for (InMemoryReservationAllocation res : reservationEntries) {
-            if (res.getEndTime() > startTime) {
-              if (user != null && !user.isEmpty()
-                      && !res.getUser().equals(user)) {
-                continue;
+      SortedMap<ReservationInterval, Set<InMemoryReservationAllocation>> res =
+          currentReservations.headMap(searchInterval, true);
+      if (!res.isEmpty()) {
+        Set<ReservationAllocation> flattenedReservations = new HashSet<>();
+        for (Set<InMemoryReservationAllocation> resEntries : res.values()) {
+          for (InMemoryReservationAllocation reservation : resEntries) {
+            // validate user
+            if (user != null && !user.isEmpty()
+                && !reservation.getUser().equals(user)) {
+              continue;
+            }
+            // handle periodic reservations
+            long period = reservation.getPeriodicity();
+            if (period > 0) {
+              long t = endTime % period;
+              // check for both contained and wrap-around reservations
+              if ((t - startTime) * (t - endTime)
+                  * (startTime - endTime) >= 0) {
+                flattenedReservations.add(reservation);
+              }
+            } else {
+              // check for non-periodic reservations
+              if (reservation.getEndTime() > startTime) {
+                flattenedReservations.add(reservation);
               }
-              flattenedReservations.add(res);
             }
           }
         }
@@ -550,36 +675,82 @@ public class InMemoryPlan implements Plan {
 
   @Override
   public RLESparseResourceAllocation getAvailableResourceOverTime(String user,
-      ReservationId oldId, long start, long end) throws PlanningException {
+      ReservationId oldId, long start, long end, long period)
+      throws PlanningException {
     readLock.lock();
     try {
-      // create RLE of totCapacity
-      TreeMap<Long, Resource> totAvailable = new TreeMap<Long, Resource>();
-      totAvailable.put(start, Resources.clone(totalCapacity));
-      RLESparseResourceAllocation totRLEAvail =
-          new RLESparseResourceAllocation(totAvailable, resCalc);
-
-      // subtract used from available
-      RLESparseResourceAllocation netAvailable;
-
-      netAvailable =
-          RLESparseResourceAllocation.merge(resCalc,
-              Resources.clone(totalCapacity), totRLEAvail, rleSparseVector,
-              RLEOperator.subtractTestNonNegative, start, end);
-
-      // add back in old reservation used resources if any
-      ReservationAllocation old = reservationTable.get(oldId);
-      if (old != null) {
-        netAvailable =
-            RLESparseResourceAllocation.merge(resCalc,
-                Resources.clone(totalCapacity), netAvailable,
-                old.getResourcesOverTime(), RLEOperator.add, start, end);
+
+      // for non-periodic return simple available resources
+      if (period == 0) {
+
+        // create RLE of totCapacity
+        TreeMap<Long, Resource> totAvailable = new TreeMap<Long, Resource>();
+        totAvailable.put(start, Resources.clone(totalCapacity));
+        RLESparseResourceAllocation totRLEAvail =
+            new RLESparseResourceAllocation(totAvailable, resCalc);
+
+        // subtract used from available
+        RLESparseResourceAllocation netAvailable;
+
+        netAvailable = RLESparseResourceAllocation.merge(resCalc,
+            Resources.clone(totalCapacity), totRLEAvail, rleSparseVector,
+            RLEOperator.subtractTestNonNegative, start, end);
+
+        // remove periodic component
+        netAvailable = RLESparseResourceAllocation.merge(resCalc,
+            Resources.clone(totalCapacity), netAvailable, periodicRle,
+            RLEOperator.subtractTestNonNegative, start, end);
+
+        // add back in old reservation used resources if any
+        ReservationAllocation old = reservationTable.get(oldId);
+        if (old != null) {
+
+          RLESparseResourceAllocation addBackPrevious =
+              old.getResourcesOverTime(start, end);
+          netAvailable = RLESparseResourceAllocation.merge(resCalc,
+              Resources.clone(totalCapacity), netAvailable, addBackPrevious,
+              RLEOperator.add, start, end);
+        }
+        // lower it if this is needed by the sharing policy
+        netAvailable = getSharingPolicy().availableResources(netAvailable, this,
+            user, oldId, start, end);
+        return netAvailable;
+      } else {
+
+        if (periodicRle.getTimePeriod() % period != 0) {
+          throw new PlanningException("The reservation periodicity (" + period
+              + ") must be" + "an exact divider of the system maxPeriod ("
+              + periodicRle.getTimePeriod() + ")");
+        }
+
+        // find the minimum resources available among all the instances that fit
+        // in the LCM
+        long numInstInLCM = periodicRle.getTimePeriod() / period;
+
+        RLESparseResourceAllocation minOverLCM =
+            getAvailableResourceOverTime(user, oldId, start, end, 0);
+        for (int i = 1; i < numInstInLCM; i++) {
+
+          long rStart = start + i * period;
+          long rEnd = end + i * period;
+
+          // recursive invocation of non-periodic range (to pick raw-info)
+          RLESparseResourceAllocation snapShot =
+              getAvailableResourceOverTime(user, oldId, rStart, rEnd, 0);
+
+          // time-align on start
+          snapShot.shift(-(i * period));
+
+          // pick the minimum amount of resources in each time interval
+          minOverLCM =
+              RLESparseResourceAllocation.merge(resCalc, getTotalCapacity(),
+                  minOverLCM, snapShot, RLEOperator.min, start, end);
+
+        }
+
+        return minOverLCM;
+
       }
-      // lower it if this is needed by the sharing policy
-      netAvailable =
-          getSharingPolicy().availableResources(netAvailable, this, user,
-              oldId, start, end);
-      return netAvailable;
     } finally {
       readLock.unlock();
     }
@@ -637,7 +808,7 @@ public class InMemoryPlan implements Plan {
   public String toCumulativeString() {
     readLock.lock();
     try {
-      return rleSparseVector.toString();
+      return rleSparseVector.toString() + "\n" + periodicRle.toString();
     } finally {
       readLock.unlock();
     }
@@ -689,11 +860,18 @@ public class InMemoryPlan implements Plan {
   }
 
   @Override
-  public RLESparseResourceAllocation getCumulativeLoadOverTime(
-      long start, long end) {
+  public RLESparseResourceAllocation getCumulativeLoadOverTime(long start,
+      long end) throws PlanningException {
     readLock.lock();
     try {
-      return rleSparseVector.getRangeOverlapping(start, end);
+
+      RLESparseResourceAllocation ret =
+          rleSparseVector.getRangeOverlapping(start, end);
+      ret = RLESparseResourceAllocation.merge(resCalc, totalCapacity, ret,
+          periodicRle.getRangeOverlapping(start, end), RLEOperator.add, start,
+          end);
+
+      return ret;
     } finally {
       readLock.unlock();
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7996eca7/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/InMemoryReservationAllocation.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/reservation/InMemoryReservationAllocation.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/InMemoryReservationAllocation.java
index 69fd43f..00c8e44 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/InMemoryReservationAllocation.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/InMemoryReservationAllocation.java
@@ -42,6 +42,7 @@ public class InMemoryReservationAllocation implements ReservationAllocation {
   private final Map<ReservationInterval, Resource> allocationRequests;
   private boolean hasGang = false;
   private long acceptedAt = -1;
+  private long periodicity = 0;
 
   private RLESparseResourceAllocation resourcesOverTime;
 
@@ -67,9 +68,16 @@ public class InMemoryReservationAllocation implements ReservationAllocation {
     this.allocationRequests = allocations;
     this.planName = planName;
     this.hasGang = hasGang;
-    resourcesOverTime = new RLESparseResourceAllocation(calculator);
-    for (Map.Entry<ReservationInterval, Resource> r : allocations
-        .entrySet()) {
+    if (contract != null && contract.getRecurrenceExpression() != null) {
+      this.periodicity = Long.parseLong(contract.getRecurrenceExpression());
+    }
+    if (periodicity > 0) {
+      resourcesOverTime =
+          new PeriodicRLESparseResourceAllocation(calculator, periodicity);
+    } else {
+      resourcesOverTime = new RLESparseResourceAllocation(calculator);
+    }
+    for (Map.Entry<ReservationInterval, Resource> r : allocations.entrySet()) {
       resourcesOverTime.addInterval(r.getKey(), r.getValue());
     }
   }
@@ -133,17 +141,33 @@ public class InMemoryReservationAllocation implements ReservationAllocation {
   }
 
   @Override
-  public RLESparseResourceAllocation getResourcesOverTime(){
+  public RLESparseResourceAllocation getResourcesOverTime() {
     return resourcesOverTime;
   }
 
   @Override
+  public RLESparseResourceAllocation getResourcesOverTime(long start,
+      long end) {
+    return resourcesOverTime.getRangeOverlapping(start, end);
+  }
+
+  @Override
+  public long getPeriodicity() {
+    return periodicity;
+  }
+
+  @Override
+  public void setPeriodicity(long period) {
+    periodicity = period;
+  }
+
+  @Override
   public String toString() {
     StringBuilder sBuf = new StringBuilder();
     sBuf.append(getReservationId()).append(" user:").append(getUser())
         .append(" startTime: ").append(getStartTime()).append(" endTime: ")
-        .append(getEndTime()).append(" alloc:\n[")
-        .append(resourcesOverTime.toString()).append("] ");
+        .append(getEndTime()).append(" Periodiciy: ").append(periodicity)
+        .append(" alloc:\n[").append(resourcesOverTime.toString()).append("] ");
     return sBuf.toString();
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7996eca7/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/NoOverCommitPolicy.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/reservation/NoOverCommitPolicy.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/NoOverCommitPolicy.java
index 55f1d00..49d4702 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/NoOverCommitPolicy.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/NoOverCommitPolicy.java
@@ -40,7 +40,7 @@ public class NoOverCommitPolicy implements SharingPolicy {
 
     RLESparseResourceAllocation available = plan.getAvailableResourceOverTime(
         reservation.getUser(), reservation.getReservationId(),
-        reservation.getStartTime(), reservation.getEndTime());
+        reservation.getStartTime(), reservation.getEndTime(), 0);
 
     // test the reservation does not exceed what is available
     try {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7996eca7/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/PeriodicRLESparseResourceAllocation.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/reservation/PeriodicRLESparseResourceAllocation.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/PeriodicRLESparseResourceAllocation.java
index 8e3be8b3..7bc44f5 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/PeriodicRLESparseResourceAllocation.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/PeriodicRLESparseResourceAllocation.java
@@ -18,47 +18,94 @@
 
 package org.apache.hadoop.yarn.server.resourcemanager.reservation;
 
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.NavigableMap;
+import java.util.TreeMap;
+
 import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.util.resource.ResourceCalculator;
 import org.apache.hadoop.yarn.util.resource.Resources;
-
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import com.google.common.annotations.VisibleForTesting;
+
 /**
- * This data structure stores a periodic RLESparseResourceAllocation.
+ * This data structure stores a periodic {@link RLESparseResourceAllocation}.
  * Default period is 1 day (86400000ms).
  */
-public class PeriodicRLESparseResourceAllocation extends
-    RLESparseResourceAllocation {
+public class PeriodicRLESparseResourceAllocation
+    extends RLESparseResourceAllocation {
 
   // Log
-  private static final Logger LOG = LoggerFactory
-      .getLogger(PeriodicRLESparseResourceAllocation.class);
+  private static final Logger LOG =
+      LoggerFactory.getLogger(PeriodicRLESparseResourceAllocation.class);
 
   private long timePeriod;
 
   /**
    * Constructor.
    *
-   * @param rleVector {@link RLESparseResourceAllocation} with the run-length
-              encoded data.
+   * @param resourceCalculator {@link ResourceCalculator} the resource
+   *          calculator to use.
    * @param timePeriod Time period in milliseconds.
    */
   public PeriodicRLESparseResourceAllocation(
-      RLESparseResourceAllocation rleVector, Long timePeriod) {
-    super(rleVector.getCumulative(), rleVector.getResourceCalculator());
+      ResourceCalculator resourceCalculator, Long timePeriod) {
+    super(resourceCalculator);
     this.timePeriod = timePeriod;
   }
 
   /**
    * Constructor. Default time period set to 1 day.
    *
+   * @param resourceCalculator {@link ResourceCalculator} the resource
+   *          calculator to use..
+   */
+  public PeriodicRLESparseResourceAllocation(
+      ResourceCalculator resourceCalculator) {
+    this(resourceCalculator,
+        YarnConfiguration.DEFAULT_RM_RESERVATION_SYSTEM_MAX_PERIODICITY);
+  }
+
+  /**
+   * Constructor.
+   *
    * @param rleVector {@link RLESparseResourceAllocation} with the run-length
-              encoded data.
+   *          encoded data.
+   * @param timePeriod Time period in milliseconds.
    */
+  @VisibleForTesting
   public PeriodicRLESparseResourceAllocation(
-      RLESparseResourceAllocation rleVector) {
-    this(rleVector, 86400000L);
+      RLESparseResourceAllocation rleVector, Long timePeriod) {
+    super(rleVector.getCumulative(), rleVector.getResourceCalculator());
+    this.timePeriod = timePeriod;
+
+    // make sure the PeriodicRLE is zero-based, and handles wrap-around
+    long delta = (getEarliestStartTime() % timePeriod - getEarliestStartTime());
+    shift(delta);
+
+    List<Long> toRemove = new ArrayList<>();
+    Map<Long, Resource> toAdd = new TreeMap<>();
+
+    for (Map.Entry<Long, Resource> entry : cumulativeCapacity.entrySet()) {
+      if (entry.getKey() > timePeriod) {
+        toRemove.add(entry.getKey());
+        if (entry.getValue() != null) {
+          toAdd.put(timePeriod, entry.getValue());
+          long prev = entry.getKey() % timePeriod;
+          toAdd.put(prev, this.getCapacityAtTime(prev));
+          toAdd.put(0L, entry.getValue());
+        }
+      }
+    }
+    for (Long l : toRemove) {
+      cumulativeCapacity.remove(l);
+    }
+    cumulativeCapacity.putAll(toAdd);
   }
 
   /**
@@ -78,24 +125,25 @@ public class PeriodicRLESparseResourceAllocation extends
    * The interval may include 0, but the end time must be strictly less than
    * timePeriod.
    *
-   * @param interval {@link ReservationInterval} to which the specified
-   *          resource is to be added.
+   * @param interval {@link ReservationInterval} to which the specified resource
+   *          is to be added.
    * @param resource {@link Resource} to be added to the interval specified.
    * @return true if addition is successful, false otherwise
    */
-  public boolean addInterval(ReservationInterval interval,
-      Resource resource) {
+  public boolean addInterval(ReservationInterval interval, Resource resource) {
     long startTime = interval.getStartTime();
     long endTime = interval.getEndTime();
+
     if (startTime >= 0 && endTime > startTime && endTime <= timePeriod) {
       return super.addInterval(interval, resource);
     } else {
-      LOG.info("Cannot set capacity beyond end time: " + timePeriod);
+      LOG.info("Cannot set capacity beyond end time: " + timePeriod + " was ("
+          + interval.toString() + ")");
       return false;
     }
   }
 
-   /**
+  /**
    * Removes a resource for the specified interval.
    *
    * @param interval the {@link ReservationInterval} for which the resource is
@@ -103,14 +151,15 @@ public class PeriodicRLESparseResourceAllocation extends
    * @param resource the {@link Resource} to be removed.
    * @return true if removal is successful, false otherwise
    */
-  public boolean removeInterval(
-      ReservationInterval interval, Resource resource) {
+  public boolean removeInterval(ReservationInterval interval,
+      Resource resource) {
     long startTime = interval.getStartTime();
     long endTime = interval.getEndTime();
     // If the resource to be subtracted is less than the minimum resource in
     // the range, abort removal to avoid negative capacity.
-    if (!Resources.fitsIn(
-        resource, super.getMinimumCapacityInInterval(interval))) {
+    // TODO revesit decrementing endTime
+    if (!Resources.fitsIn(resource, getMinimumCapacityInInterval(
+        new ReservationInterval(startTime, endTime - 1)))) {
       LOG.info("Request to remove more resources than what is available");
       return false;
     }
@@ -125,17 +174,16 @@ public class PeriodicRLESparseResourceAllocation extends
   /**
    * Get maximum capacity at periodic offsets from the specified time.
    *
-   * @param tick UTC time base from which offsets are specified for finding
-   *          the maximum capacity.
-   * @param period periodic offset at which capacities are evaluted.
+   * @param tick UTC time base from which offsets are specified for finding the
+   *          maximum capacity.
+   * @param period periodic offset at which capacities are evaluated.
    * @return the maximum {@link Resource} across the specified time instants.
    * @return true if removal is successful, false otherwise
    */
   public Resource getMaximumPeriodicCapacity(long tick, long period) {
     Resource maxResource;
     if (period < timePeriod) {
-      maxResource =
-          super.getMaximumPeriodicCapacity(tick % timePeriod, period);
+      maxResource = super.getMaximumPeriodicCapacity(tick % timePeriod, period);
     } else {
       // if period is greater than the length of PeriodicRLESparseAllocation,
       // only a single value exists in this interval.
@@ -164,4 +212,30 @@ public class PeriodicRLESparseResourceAllocation extends
     return ret.toString();
   }
 
+  @Override
+  public RLESparseResourceAllocation getRangeOverlapping(long start, long end) {
+    NavigableMap<Long, Resource> unrolledMap = new TreeMap<>();
+    readLock.lock();
+    try {
+      long relativeStart = (start >= 0) ? start % timePeriod : 0;
+      NavigableMap<Long, Resource> cumulativeMap = this.getCumulative();
+      Long previous = cumulativeMap.floorKey(relativeStart);
+      previous = (previous != null) ? previous : 0;
+      for (long i = 0; i <= (end - start) / timePeriod; i++) {
+        for (Map.Entry<Long, Resource> e : cumulativeMap.entrySet()) {
+          long curKey = e.getKey() + (i * timePeriod);
+          if (curKey >= previous && (start + curKey - relativeStart) <= end) {
+            unrolledMap.put(curKey, e.getValue());
+          }
+        }
+      }
+      RLESparseResourceAllocation rle =
+          new RLESparseResourceAllocation(unrolledMap, getResourceCalculator());
+      rle.shift(start - relativeStart);
+      return rle;
+    } finally {
+      readLock.unlock();
+    }
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7996eca7/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/PlanEdit.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/reservation/PlanEdit.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/PlanEdit.java
index 504a250..9afa324 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/PlanEdit.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/PlanEdit.java
@@ -28,54 +28,58 @@ import org.apache.hadoop.yarn.server.resourcemanager.reservation.exceptions.Plan
 public interface PlanEdit extends PlanContext, PlanView {
 
   /**
-   * Add a new {@link ReservationAllocation} to the plan
+   * Add a new {@link ReservationAllocation} to the plan.
    * 
    * @param reservation the {@link ReservationAllocation} to be added to the
    *          plan
    * @param isRecovering flag to indicate if reservation is being added as part
    *          of failover or not
    * @return true if addition is successful, false otherwise
+   * @throws PlanningException if addition is unsuccessful
    */
-  public boolean addReservation(ReservationAllocation reservation,
+  boolean addReservation(ReservationAllocation reservation,
       boolean isRecovering) throws PlanningException;
 
   /**
    * Updates an existing {@link ReservationAllocation} in the plan. This is
-   * required for re-negotiation
+   * required for re-negotiation.
    * 
    * @param reservation the {@link ReservationAllocation} to be updated the plan
    * @return true if update is successful, false otherwise
+   * @throws PlanningException if update is unsuccessful
    */
-  public boolean updateReservation(ReservationAllocation reservation)
+  boolean updateReservation(ReservationAllocation reservation)
       throws PlanningException;
 
   /**
    * Delete an existing {@link ReservationAllocation} from the plan identified
    * uniquely by its {@link ReservationId}. This will generally be used for
-   * garbage collection
+   * garbage collection.
    * 
    * @param reservationID the {@link ReservationAllocation} to be deleted from
    *          the plan identified uniquely by its {@link ReservationId}
    * @return true if delete is successful, false otherwise
+   * @throws PlanningException if deletion is unsuccessful
    */
-  public boolean deleteReservation(ReservationId reservationID)
+  boolean deleteReservation(ReservationId reservationID)
       throws PlanningException;
 
   /**
    * Method invoked to garbage collect old reservations. It cleans up expired
-   * reservations that have fallen out of the sliding archival window
+   * reservations that have fallen out of the sliding archival window.
    * 
    * @param tick the current time from which the archival window is computed
+   * @throws PlanningException if archival is unsuccessful
    */
-  public void archiveCompletedReservations(long tick) throws PlanningException;
+  void archiveCompletedReservations(long tick) throws PlanningException;
 
   /**
    * Sets the overall capacity in terms of {@link Resource} assigned to this
-   * plan
+   * plan.
    * 
    * @param capacity the overall capacity in terms of {@link Resource} assigned
    *          to this plan
    */
-  public void setTotalCapacity(Resource capacity);
+  void setTotalCapacity(Resource capacity);
 
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7996eca7/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/PlanView.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/reservation/PlanView.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/PlanView.java
index 2767993..4035f68 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/PlanView.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/PlanView.java
@@ -17,50 +17,50 @@
  */
 package org.apache.hadoop.yarn.server.resourcemanager.reservation;
 
+import java.util.Set;
+
 import org.apache.hadoop.yarn.api.records.ReservationId;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.server.resourcemanager.reservation.exceptions.PlanningException;
 
-import java.util.Set;
-
 /**
  * This interface provides a read-only view on the allocations made in this
  * plan. This methods are used for example by {@code ReservationAgent}s to
  * determine the free resources in a certain point in time, and by
  * PlanFollowerPolicy to publish this plan to the scheduler.
  */
-public interface PlanView extends PlanContext {
+interface PlanView extends PlanContext {
 
   /**
    * Return a set of {@link ReservationAllocation} identified by the user who
    * made the reservation.
    *
    * @param reservationID the unqiue id to identify the
-   * {@link ReservationAllocation}
+   *          {@link ReservationAllocation}
    * @param interval the time interval used to retrieve the reservation
-   *                 allocations from. Only reservations with start time no
-   *                 greater than the interval end time, and end time no less
-   *                 than the interval start time will be selected.
+   *          allocations from. Only reservations with start time no greater
+   *          than the interval end time, and end time no less than the interval
+   *          start time will be selected.
    * @param user the user to retrieve the reservation allocation from.
    * @return a set of {@link ReservationAllocation} identified by the user who
-   * made the reservation
+   *         made the reservation
    */
-  Set<ReservationAllocation> getReservations(ReservationId
-                    reservationID, ReservationInterval interval, String user);
+  Set<ReservationAllocation> getReservations(ReservationId reservationID,
+      ReservationInterval interval, String user);
 
   /**
    * Return a set of {@link ReservationAllocation} identified by any user.
    *
    * @param reservationID the unqiue id to identify the
-   * {@link ReservationAllocation}
+   *          {@link ReservationAllocation}
    * @param interval the time interval used to retrieve the reservation
-   *                 allocations from. Only reservations with start time no
-   *                 greater than the interval end time, and end time no less
-   *                 than the interval start time will be selected.
+   *          allocations from. Only reservations with start time no greater
+   *          than the interval end time, and end time no less than the interval
+   *          start time will be selected.
    * @return a set of {@link ReservationAllocation} identified by any user
    */
   Set<ReservationAllocation> getReservations(ReservationId reservationID,
-                    ReservationInterval interval);
+      ReservationInterval interval);
 
   /**
    * Return a {@link ReservationAllocation} identified by its
@@ -70,7 +70,7 @@ public interface PlanView extends PlanContext {
    *          {@link ReservationAllocation}
    * @return {@link ReservationAllocation} identified by the specified id
    */
-  public ReservationAllocation getReservationById(ReservationId reservationID);
+  ReservationAllocation getReservationById(ReservationId reservationID);
 
   /**
    * Return a set of {@link ReservationAllocation} that belongs to a certain
@@ -78,11 +78,10 @@ public interface PlanView extends PlanContext {
    *
    * @param user the user being considered
    * @param t the instant in time being considered
-   * @return set of active {@link ReservationAllocation}s for this
-   *         user at this time
+   * @return set of active {@link ReservationAllocation}s for this user at this
+   *         time
    */
-  public Set<ReservationAllocation> getReservationByUserAtTime(String user,
-      long t);
+  Set<ReservationAllocation> getReservationByUserAtTime(String user, long t);
 
   /**
    * Gets all the active reservations at the specified point of time
@@ -91,14 +90,14 @@ public interface PlanView extends PlanContext {
    *          requested
    * @return set of active reservations at the specified time
    */
-  public Set<ReservationAllocation> getReservationsAtTime(long tick);
+  Set<ReservationAllocation> getReservationsAtTime(long tick);
 
   /**
    * Gets all the reservations in the plan
    * 
    * @return set of all reservations handled by this Plan
    */
-  public Set<ReservationAllocation> getAllReservations();
+  Set<ReservationAllocation> getAllReservations();
 
   /**
    * Returns the total {@link Resource} reserved for all users at the specified
@@ -126,61 +125,68 @@ public interface PlanView extends PlanContext {
    * 
    * @return the time (UTC in ms) at which the first reservation starts
    */
-  public long getEarliestStartTime();
+  long getEarliestStartTime();
 
   /**
    * Returns the time (UTC in ms) at which the last reservation terminates
    *
    * @return the time (UTC in ms) at which the last reservation terminates
    */
-  public long getLastEndTime();
+  long getLastEndTime();
 
   /**
    * This method returns the amount of resources available to a given user
    * (optionally if removing a certain reservation) over the start-end time
-   * range.
+   * range. If the request is periodic (period is non-zero) we return the
+   * minimum amount of resources available to periodic reservations (in all
+   * "period" windows within the system maxPeriod / LCM).
    *
-   * @param user
-   * @param oldId
-   * @param start
-   * @param end
+   * @param user the user being considered
+   * @param oldId the identifier of the existing reservation
+   * @param start start of the time interval.
+   * @param end end of the time interval.
+   * @param period the ms periodicty for this request (loop and pick min till
+   *          maxPeriodicity)
    * @return a view of the plan as it is available to this user
-   * @throws PlanningException
+   * @throws PlanningException if operation is unsuccessful
    */
-  public RLESparseResourceAllocation getAvailableResourceOverTime(String user,
-      ReservationId oldId, long start, long end) throws PlanningException;
+  RLESparseResourceAllocation getAvailableResourceOverTime(String user,
+      ReservationId oldId, long start, long end, long period)
+      throws PlanningException;
 
   /**
    * This method returns a RLE encoded view of the user reservation count
    * utilization between start and end time.
    *
-   * @param user
-   * @param start
-   * @param end
+   * @param user the user being considered
+   * @param start start of the time interval.
+   * @param end end of the time interval.
    * @return RLE encoded view of reservation used over time
    */
-  public RLESparseResourceAllocation getReservationCountForUserOverTime(
-      String user, long start, long end);
+  RLESparseResourceAllocation getReservationCountForUserOverTime(String user,
+      long start, long end);
 
   /**
    * This method returns a RLE encoded view of the user reservation utilization
    * between start and end time.
    *
-   * @param user
-   * @param start
-   * @param end
+   * @param user the user being considered
+   * @param start start of the time interval.
+   * @param end end of the time interval.
    * @return RLE encoded view of resources used over time
    */
-  public RLESparseResourceAllocation getConsumptionForUserOverTime(String user,
+  RLESparseResourceAllocation getConsumptionForUserOverTime(String user,
       long start, long end);
 
   /**
    * Get the cumulative load over a time interval.
    *
-   * @param start Start of the time interval.
-   * @param end End of the time interval.
+   * @param start start of the time interval.
+   * @param end end of the time interval.
    * @return RLE sparse allocation.
+   * @throws PlanningException if operation is unsuccessful
    */
-  RLESparseResourceAllocation getCumulativeLoadOverTime(long start, long end);
+  RLESparseResourceAllocation getCumulativeLoadOverTime(long start, long end)
+      throws PlanningException;
 
 }


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[31/37] hadoop git commit: HDFS-12357. Let NameNode to bypass external attribute provider for configured users. Contributed by Yongjun Zhang, Arun Suresh.

Posted by ae...@apache.org.
HDFS-12357. Let NameNode to bypass external attribute provider for configured users. Contributed by Yongjun Zhang, Arun Suresh.


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

Branch: refs/heads/HDFS-7240
Commit: d77ed238a911fc85d6f4bbce606cac7ec44f557f
Parents: 5ff7416
Author: Yongjun Zhang <yz...@cloudera.com>
Authored: Thu Sep 7 09:45:45 2017 -0700
Committer: Yongjun Zhang <yz...@cloudera.com>
Committed: Thu Sep 7 09:50:36 2017 -0700

----------------------------------------------------------------------
 .../org/apache/hadoop/hdfs/DFSConfigKeys.java   |   3 +
 .../hdfs/server/namenode/FSDirectory.java       |  58 +++++++++-
 .../src/main/resources/hdfs-default.xml         |  12 +++
 .../namenode/TestINodeAttributeProvider.java    | 105 +++++++++++++++++--
 4 files changed, 167 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/d77ed238/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
index bc7b716..d06e378 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
@@ -641,6 +641,9 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
   public static final String  DFS_DATANODE_MIN_SUPPORTED_NAMENODE_VERSION_KEY = "dfs.datanode.min.supported.namenode.version";
   public static final String  DFS_DATANODE_MIN_SUPPORTED_NAMENODE_VERSION_DEFAULT = "2.1.0-beta";
   public static final String  DFS_NAMENODE_INODE_ATTRIBUTES_PROVIDER_KEY = "dfs.namenode.inode.attributes.provider.class";
+  public static final String  DFS_NAMENODE_INODE_ATTRIBUTES_PROVIDER_BYPASS_USERS_KEY = "dfs.namenode.inode.attributes.provider.bypass.users";
+  public static final String  DFS_NAMENODE_INODE_ATTRIBUTES_PROVIDER_BYPASS_USERS_DEFAULT = "";
+
   public static final String  DFS_DATANODE_BP_READY_TIMEOUT_KEY = "dfs.datanode.bp-ready.timeout";
   public static final long    DFS_DATANODE_BP_READY_TIMEOUT_DEFAULT = 20;
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d77ed238/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
index e6aa533..6604b5a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
@@ -74,6 +74,7 @@ import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
 import java.util.EnumSet;
+import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
 import java.util.SortedSet;
@@ -202,6 +203,10 @@ public class FSDirectory implements Closeable {
 
   private INodeAttributeProvider attributeProvider;
 
+  // A HashSet of principals of users for whom the external attribute provider
+  // will be bypassed
+  private HashSet<String> usersToBypassExtAttrProvider = null;
+
   public void setINodeAttributeProvider(INodeAttributeProvider provider) {
     attributeProvider = provider;
   }
@@ -357,6 +362,49 @@ public class FSDirectory implements Closeable {
     this.quotaInitThreads = conf.getInt(
         DFSConfigKeys.DFS_NAMENODE_QUOTA_INIT_THREADS_KEY,
         DFSConfigKeys.DFS_NAMENODE_QUOTA_INIT_THREADS_DEFAULT);
+
+    initUsersToBypassExtProvider(conf);
+  }
+
+  private void initUsersToBypassExtProvider(Configuration conf) {
+    String[] bypassUsers = conf.getTrimmedStrings(
+        DFSConfigKeys.DFS_NAMENODE_INODE_ATTRIBUTES_PROVIDER_BYPASS_USERS_KEY,
+        DFSConfigKeys.DFS_NAMENODE_INODE_ATTRIBUTES_PROVIDER_BYPASS_USERS_DEFAULT);
+    for(int i = 0; i < bypassUsers.length; i++) {
+      String tmp = bypassUsers[i].trim();
+      if (!tmp.isEmpty()) {
+        if (usersToBypassExtAttrProvider == null) {
+          usersToBypassExtAttrProvider = new HashSet<String>();
+        }
+        LOG.info("Add user " + tmp + " to the list that will bypass external"
+            + " attribute provider.");
+        usersToBypassExtAttrProvider.add(tmp);
+      }
+    }
+  }
+
+  /**
+   * Check if a given user is configured to bypass external attribute provider.
+   * @param user user principal
+   * @return true if the user is to bypass external attribute provider
+   */
+  private boolean isUserBypassingExtAttrProvider(final String user) {
+    return (usersToBypassExtAttrProvider != null) &&
+          usersToBypassExtAttrProvider.contains(user);
+  }
+
+  /**
+   * Return attributeProvider or null if ugi is to bypass attributeProvider.
+   * @param ugi
+   * @return configured attributeProvider or null
+   */
+  private INodeAttributeProvider getUserFilteredAttributeProvider(
+      UserGroupInformation ugi) {
+    if (attributeProvider == null ||
+        (ugi != null && isUserBypassingExtAttrProvider(ugi.getUserName()))) {
+      return null;
+    }
+    return attributeProvider;
   }
 
   /**
@@ -1711,7 +1759,7 @@ public class FSDirectory implements Closeable {
   FSPermissionChecker getPermissionChecker(String fsOwner, String superGroup,
       UserGroupInformation ugi) throws AccessControlException {
     return new FSPermissionChecker(
-        fsOwner, superGroup, ugi, attributeProvider);
+        fsOwner, superGroup, ugi, getUserFilteredAttributeProvider(ugi));
   }
 
   void checkOwner(FSPermissionChecker pc, INodesInPath iip)
@@ -1896,18 +1944,20 @@ public class FSDirectory implements Closeable {
   }
 
   INodeAttributes getAttributes(INodesInPath iip)
-      throws FileNotFoundException {
+      throws IOException {
     INode node = FSDirectory.resolveLastINode(iip);
     int snapshot = iip.getPathSnapshotId();
     INodeAttributes nodeAttrs = node.getSnapshotINode(snapshot);
-    if (attributeProvider != null) {
+    UserGroupInformation ugi = NameNode.getRemoteUser();
+    INodeAttributeProvider ap = this.getUserFilteredAttributeProvider(ugi);
+    if (ap != null) {
       // permission checking sends the full components array including the
       // first empty component for the root.  however file status
       // related calls are expected to strip out the root component according
       // to TestINodeAttributeProvider.
       byte[][] components = iip.getPathComponents();
       components = Arrays.copyOfRange(components, 1, components.length);
-      nodeAttrs = attributeProvider.getAttributes(components, nodeAttrs);
+      nodeAttrs = ap.getAttributes(components, nodeAttrs);
     }
     return nodeAttrs;
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d77ed238/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 d58e54e..36c74f6 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
@@ -4160,6 +4160,18 @@
 </property>
 
 <property>
+  <name>dfs.namenode.authorization.provider.bypass.users</name>
+  <value></value>
+  <description>
+    A list of user principals (in secure cluster) or user names (in insecure
+    cluster) for whom the external attribute provider will be bypassed for all
+    operations. This means file attributes stored in HDFS instead of the
+    external provider will be used for permission checking and be returned when
+    requested.
+  </description>
+</property>
+
+<property>
   <name>dfs.namenode.max-num-blocks-to-log</name>
   <value>1000</value>
   <description>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d77ed238/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestINodeAttributeProvider.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestINodeAttributeProvider.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestINodeAttributeProvider.java
index ffdc535..bbc5fa0 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestINodeAttributeProvider.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestINodeAttributeProvider.java
@@ -24,6 +24,7 @@ import java.util.Map;
 import java.util.Set;
 
 import com.google.common.collect.ImmutableList;
+
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
@@ -33,19 +34,25 @@ import org.apache.hadoop.fs.permission.*;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
-import org.apache.hadoop.hdfs.server.namenode.INodeAttributeProvider.AccessControlEnforcer;
 import org.apache.hadoop.security.AccessControlException;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.junit.After;
 import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import com.google.common.collect.Lists;
 
 public class TestINodeAttributeProvider {
+  private static final Logger LOG =
+      LoggerFactory.getLogger(TestINodeAttributeProvider.class);
+
   private MiniDFSCluster miniDFS;
   private static final Set<String> CALLED = new HashSet<String>();
+  private static final short HDFS_PERMISSION = 0777;
+  private static final short PROVIDER_PERMISSION = 0770;
 
   public static class MyAuthorizationProvider extends INodeAttributeProvider {
 
@@ -112,7 +119,8 @@ public class TestINodeAttributeProvider {
 
         @Override
         public long getPermissionLong() {
-          return (useDefault) ? inode.getPermissionLong() : 0770;
+          return (useDefault) ? inode.getPermissionLong() :
+            (long)PROVIDER_PERMISSION;
         }
 
         @Override
@@ -177,6 +185,9 @@ public class TestINodeAttributeProvider {
     conf.set(DFSConfigKeys.DFS_NAMENODE_INODE_ATTRIBUTES_PROVIDER_KEY,
         MyAuthorizationProvider.class.getName());
     conf.setBoolean(DFSConfigKeys.DFS_NAMENODE_ACLS_ENABLED_KEY, true);
+    conf.set(
+        DFSConfigKeys.DFS_NAMENODE_INODE_ATTRIBUTES_PROVIDER_BYPASS_USERS_KEY,
+        " u2,, ,u3, ");
     EditLogFileOutputStream.setShouldSkipFsyncForTesting(true);
     miniDFS = new MiniDFSCluster.Builder(conf).build();
   }
@@ -195,8 +206,11 @@ public class TestINodeAttributeProvider {
   public void testDelegationToProvider() throws Exception {
     Assert.assertTrue(CALLED.contains("start"));
     FileSystem fs = FileSystem.get(miniDFS.getConfiguration(0));
-    fs.mkdirs(new Path("/tmp"));
-    fs.setPermission(new Path("/tmp"), new FsPermission((short) 0777));
+    final Path tmpPath = new Path("/tmp");
+    final Path fooPath = new Path("/tmp/foo");
+
+    fs.mkdirs(tmpPath);
+    fs.setPermission(tmpPath, new FsPermission(HDFS_PERMISSION));
     UserGroupInformation ugi = UserGroupInformation.createUserForTesting("u1",
         new String[]{"g1"});
     ugi.doAs(new PrivilegedExceptionAction<Void>() {
@@ -204,17 +218,19 @@ public class TestINodeAttributeProvider {
       public Void run() throws Exception {
         FileSystem fs = FileSystem.get(miniDFS.getConfiguration(0));
         CALLED.clear();
-        fs.mkdirs(new Path("/tmp/foo"));
+        fs.mkdirs(fooPath);
         Assert.assertTrue(CALLED.contains("getAttributes"));
         Assert.assertTrue(CALLED.contains("checkPermission|null|null|null"));
         Assert.assertTrue(CALLED.contains("checkPermission|WRITE|null|null"));
+
         CALLED.clear();
-        fs.listStatus(new Path("/tmp/foo"));
+        fs.listStatus(fooPath);
         Assert.assertTrue(CALLED.contains("getAttributes"));
         Assert.assertTrue(
             CALLED.contains("checkPermission|null|null|READ_EXECUTE"));
+
         CALLED.clear();
-        fs.getAclStatus(new Path("/tmp/foo"));
+        fs.getAclStatus(fooPath);
         Assert.assertTrue(CALLED.contains("getAttributes"));
         Assert.assertTrue(CALLED.contains("checkPermission|null|null|null"));
         return null;
@@ -222,6 +238,81 @@ public class TestINodeAttributeProvider {
     });
   }
 
+  private class AssertHelper {
+    private boolean bypass = true;
+    AssertHelper(boolean bp) {
+      bypass = bp;
+    }
+    public void doAssert(boolean x) {
+      if (bypass) {
+        Assert.assertFalse(x);
+      } else {
+        Assert.assertTrue(x);
+      }
+    }
+  }
+
+  private void testBypassProviderHelper(final String[] users,
+      final short expectedPermission, final boolean bypass) throws Exception {
+    final AssertHelper asserter = new AssertHelper(bypass);
+
+    Assert.assertTrue(CALLED.contains("start"));
+
+    FileSystem fs = FileSystem.get(miniDFS.getConfiguration(0));
+    final Path userPath = new Path("/user");
+    final Path authz = new Path("/user/authz");
+    final Path authzChild = new Path("/user/authz/child2");
+
+    fs.mkdirs(userPath);
+    fs.setPermission(userPath, new FsPermission(HDFS_PERMISSION));
+    fs.mkdirs(authz);
+    fs.setPermission(authz, new FsPermission(HDFS_PERMISSION));
+    fs.mkdirs(authzChild);
+    fs.setPermission(authzChild, new FsPermission(HDFS_PERMISSION));
+    for(String user : users) {
+      UserGroupInformation ugiBypass =
+          UserGroupInformation.createUserForTesting(user,
+              new String[]{"g1"});
+      ugiBypass.doAs(new PrivilegedExceptionAction<Void>() {
+        @Override
+        public Void run() throws Exception {
+          FileSystem fs = FileSystem.get(miniDFS.getConfiguration(0));
+          Assert.assertEquals(expectedPermission,
+              fs.getFileStatus(authzChild).getPermission().toShort());
+          asserter.doAssert(CALLED.contains("getAttributes"));
+          asserter.doAssert(CALLED.contains("checkPermission|null|null|null"));
+
+          CALLED.clear();
+          Assert.assertEquals(expectedPermission,
+              fs.listStatus(userPath)[0].getPermission().toShort());
+          asserter.doAssert(CALLED.contains("getAttributes"));
+          asserter.doAssert(
+              CALLED.contains("checkPermission|null|null|READ_EXECUTE"));
+
+          CALLED.clear();
+          fs.getAclStatus(authzChild);
+          asserter.doAssert(CALLED.contains("getAttributes"));
+          asserter.doAssert(CALLED.contains("checkPermission|null|null|null"));
+          return null;
+        }
+      });
+    }
+  }
+
+  @Test
+  public void testAuthzDelegationToProvider() throws Exception {
+    LOG.info("Test not bypassing provider");
+    String[] users = {"u1"};
+    testBypassProviderHelper(users, PROVIDER_PERMISSION, false);
+  }
+
+  @Test
+  public void testAuthzBypassingProvider() throws Exception {
+    LOG.info("Test bypassing provider");
+    String[] users = {"u2", "u3"};
+    testBypassProviderHelper(users, HDFS_PERMISSION, true);
+  }
+
   @Test
   public void testCustomProvider() throws Exception {
     FileSystem fs = FileSystem.get(miniDFS.getConfiguration(0));


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[12/37] hadoop git commit: HADOOP-14688. Intern strings in KeyVersion and EncryptedKeyVersion. Contributed by Xiao Chen.

Posted by ae...@apache.org.
HADOOP-14688. Intern strings in KeyVersion and EncryptedKeyVersion. Contributed by Xiao Chen.


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

Branch: refs/heads/HDFS-7240
Commit: ad32759fd9f33e7bd18758ad1a5a464dab3bcbd9
Parents: ccd2ac6
Author: Wei-Chiu Chuang <we...@apache.org>
Authored: Tue Sep 5 15:11:37 2017 -0700
Committer: Wei-Chiu Chuang <we...@apache.org>
Committed: Tue Sep 5 15:11:37 2017 -0700

----------------------------------------------------------------------
 .../src/main/java/org/apache/hadoop/crypto/key/KeyProvider.java | 4 ++--
 .../apache/hadoop/crypto/key/KeyProviderCryptoExtension.java    | 5 +++--
 2 files changed, 5 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/ad32759f/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/KeyProvider.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/KeyProvider.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/KeyProvider.java
index fb3dba4..c1c371b 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/KeyProvider.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/KeyProvider.java
@@ -71,8 +71,8 @@ public abstract class KeyProvider {
 
     protected KeyVersion(String name, String versionName,
                          byte[] material) {
-      this.name = name;
-      this.versionName = versionName;
+      this.name = name == null ? null : name.intern();
+      this.versionName = versionName == null ? null : versionName.intern();
       this.material = material;
     }
     

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ad32759f/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/KeyProviderCryptoExtension.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/KeyProviderCryptoExtension.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/KeyProviderCryptoExtension.java
index 8c879b3..693c785 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/KeyProviderCryptoExtension.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/KeyProviderCryptoExtension.java
@@ -81,8 +81,9 @@ public class KeyProviderCryptoExtension extends
     protected EncryptedKeyVersion(String keyName,
         String encryptionKeyVersionName, byte[] encryptedKeyIv,
         KeyVersion encryptedKeyVersion) {
-      this.encryptionKeyName = keyName;
-      this.encryptionKeyVersionName = encryptionKeyVersionName;
+      this.encryptionKeyName = keyName == null ? null : keyName.intern();
+      this.encryptionKeyVersionName = encryptionKeyVersionName == null ?
+          null : encryptionKeyVersionName.intern();
       this.encryptedKeyIv = encryptedKeyIv;
       this.encryptedKeyVersion = encryptedKeyVersion;
     }


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[27/37] hadoop git commit: YARN-5328. Plan/ResourceAllocation data structure enhancements required to support recurring reservations in ReservationSystem.

Posted by ae...@apache.org.
YARN-5328. Plan/ResourceAllocation data structure enhancements required to support recurring reservations in ReservationSystem.


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

Branch: refs/heads/HDFS-7240
Commit: b6e7d1369690eaf50ce9ea7968f91a72ecb74de0
Parents: e3345e9
Author: Subru Krishnan <su...@apache.org>
Authored: Wed Sep 6 16:46:01 2017 -0700
Committer: Subru Krishnan <su...@apache.org>
Committed: Wed Sep 6 16:46:01 2017 -0700

----------------------------------------------------------------------
 .../hadoop/yarn/conf/YarnConfiguration.java     |   6 +
 .../yarn/conf/TestYarnConfigurationFields.java  |   4 +-
 .../reservation/AbstractReservationSystem.java  |  90 ++--
 .../AbstractSchedulerPlanFollower.java          | 183 ++++----
 .../reservation/InMemoryPlan.java               | 400 ++++++++++++-----
 .../InMemoryReservationAllocation.java          |  36 +-
 .../reservation/NoOverCommitPolicy.java         |   2 +-
 .../PeriodicRLESparseResourceAllocation.java    | 130 ++++--
 .../resourcemanager/reservation/PlanEdit.java   |  24 +-
 .../resourcemanager/reservation/PlanView.java   |  94 ++--
 .../RLESparseResourceAllocation.java            | 112 ++---
 .../reservation/ReservationAllocation.java      |  60 ++-
 .../reservation/ReservationInputValidator.java  | 134 +++---
 .../reservation/ReservationSystem.java          |   6 +-
 .../reservation/SharingPolicy.java              |  13 +-
 .../reservation/planning/Planner.java           |   2 +-
 .../reservation/planning/PlanningAlgorithm.java |   2 +-
 .../reservation/planning/StageAllocator.java    |   2 +-
 .../planning/StageAllocatorGreedy.java          |   2 +-
 .../planning/StageAllocatorGreedyRLE.java       |   5 +-
 .../planning/StageAllocatorLowCostAligned.java  |   4 +-
 .../reservation/ReservationSystemTestUtil.java  | 135 +++---
 .../reservation/TestInMemoryPlan.java           | 431 ++++++++++++-------
 ...TestPeriodicRLESparseResourceAllocation.java | 109 +++--
 .../TestRLESparseResourceAllocation.java        | 122 +++---
 .../planning/TestSimpleCapacityReplanner.java   |   8 +-
 26 files changed, 1340 insertions(+), 776 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/b6e7d136/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
index 4944821..27ca957 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
@@ -262,6 +262,12 @@ public class YarnConfiguration extends Configuration {
   public static final long DEFAULT_RM_RESERVATION_SYSTEM_PLAN_FOLLOWER_TIME_STEP =
       1000L;
 
+  /** The maximum periodicity for the Reservation System. */
+  public static final String RM_RESERVATION_SYSTEM_MAX_PERIODICITY =
+      RM_PREFIX + "reservation-system.max-periodicity";
+  public static final long DEFAULT_RM_RESERVATION_SYSTEM_MAX_PERIODICITY =
+      86400000L;
+
   /**
    * Enable periodic monitor threads.
    * @see #RM_SCHEDULER_MONITOR_POLICIES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b6e7d136/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/test/java/org/apache/hadoop/yarn/conf/TestYarnConfigurationFields.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/test/java/org/apache/hadoop/yarn/conf/TestYarnConfigurationFields.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/test/java/org/apache/hadoop/yarn/conf/TestYarnConfigurationFields.java
index bd7bf93..1d3111c 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/test/java/org/apache/hadoop/yarn/conf/TestYarnConfigurationFields.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/test/java/org/apache/hadoop/yarn/conf/TestYarnConfigurationFields.java
@@ -33,7 +33,7 @@ import org.apache.hadoop.conf.TestConfigurationFieldsBase;
  */
 public class TestYarnConfigurationFields extends TestConfigurationFieldsBase {
 
-  @SuppressWarnings("deprecation")
+  @SuppressWarnings({"deprecation", "methodlength"})
   @Override
   public void initializeMemberVariables() {
     xmlFilename = new String("yarn-default.xml");
@@ -69,6 +69,8 @@ public class TestYarnConfigurationFields extends TestConfigurationFieldsBase {
     configurationPropsToSkipCompare.add(YarnConfiguration
         .YARN_SECURITY_SERVICE_AUTHORIZATION_COLLECTOR_NODEMANAGER_PROTOCOL);
     configurationPropsToSkipCompare.add(YarnConfiguration.CURATOR_LEADER_ELECTOR);
+    configurationPropsToSkipCompare
+        .add(YarnConfiguration.RM_RESERVATION_SYSTEM_MAX_PERIODICITY);
 
     // Federation default configs to be ignored
     configurationPropsToSkipCompare

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b6e7d136/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/AbstractReservationSystem.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/reservation/AbstractReservationSystem.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/AbstractReservationSystem.java
index 5ef4912..5b8772c 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/AbstractReservationSystem.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/AbstractReservationSystem.java
@@ -18,6 +18,17 @@
 
 package org.apache.hadoop.yarn.server.resourcemanager.reservation;
 
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Set;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+
 import org.apache.hadoop.classification.InterfaceAudience.LimitedPrivate;
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.conf.Configuration;
@@ -46,17 +57,6 @@ import org.apache.hadoop.yarn.util.resource.ResourceCalculator;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import java.util.HashMap;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.Set;
-import java.util.concurrent.ScheduledExecutorService;
-import java.util.concurrent.ScheduledThreadPoolExecutor;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicLong;
-import java.util.concurrent.locks.Lock;
-import java.util.concurrent.locks.ReentrantReadWriteLock;
-
 /**
  * This is the implementation of {@link ReservationSystem} based on the
  * {@link ResourceScheduler}
@@ -66,8 +66,8 @@ import java.util.concurrent.locks.ReentrantReadWriteLock;
 public abstract class AbstractReservationSystem extends AbstractService
     implements ReservationSystem {
 
-  private static final Logger LOG = LoggerFactory
-      .getLogger(AbstractReservationSystem.class);
+  private static final Logger LOG =
+      LoggerFactory.getLogger(AbstractReservationSystem.class);
 
   // private static final String DEFAULT_CAPACITY_SCHEDULER_PLAN
 
@@ -103,6 +103,8 @@ public abstract class AbstractReservationSystem extends AbstractService
 
   private boolean isRecoveryEnabled = false;
 
+  private long maxPeriodicity;
+
   /**
    * Construct the service.
    * 
@@ -143,36 +145,41 @@ public abstract class AbstractReservationSystem extends AbstractService
     this.conf = conf;
     scheduler = rmContext.getScheduler();
     // Get the plan step size
-    planStepSize =
-        conf.getTimeDuration(
-            YarnConfiguration.RM_RESERVATION_SYSTEM_PLAN_FOLLOWER_TIME_STEP,
-            YarnConfiguration.DEFAULT_RM_RESERVATION_SYSTEM_PLAN_FOLLOWER_TIME_STEP,
-            TimeUnit.MILLISECONDS);
+    planStepSize = conf.getTimeDuration(
+        YarnConfiguration.RM_RESERVATION_SYSTEM_PLAN_FOLLOWER_TIME_STEP,
+        YarnConfiguration.DEFAULT_RM_RESERVATION_SYSTEM_PLAN_FOLLOWER_TIME_STEP,
+        TimeUnit.MILLISECONDS);
     if (planStepSize < 0) {
       planStepSize =
           YarnConfiguration.DEFAULT_RM_RESERVATION_SYSTEM_PLAN_FOLLOWER_TIME_STEP;
     }
+    maxPeriodicity =
+        conf.getLong(YarnConfiguration.RM_RESERVATION_SYSTEM_MAX_PERIODICITY,
+            YarnConfiguration.DEFAULT_RM_RESERVATION_SYSTEM_MAX_PERIODICITY);
+    if (maxPeriodicity <= 0) {
+      maxPeriodicity =
+          YarnConfiguration.DEFAULT_RM_RESERVATION_SYSTEM_MAX_PERIODICITY;
+    }
     // Create a plan corresponding to every reservable queue
     Set<String> planQueueNames = scheduler.getPlanQueues();
     for (String planQueueName : planQueueNames) {
       Plan plan = initializePlan(planQueueName);
       plans.put(planQueueName, plan);
     }
-    isRecoveryEnabled = conf.getBoolean(
-        YarnConfiguration.RECOVERY_ENABLED,
+    isRecoveryEnabled = conf.getBoolean(YarnConfiguration.RECOVERY_ENABLED,
         YarnConfiguration.DEFAULT_RM_RECOVERY_ENABLED);
 
     if (conf.getBoolean(YarnConfiguration.YARN_RESERVATION_ACL_ENABLE,
-            YarnConfiguration.DEFAULT_YARN_RESERVATION_ACL_ENABLE) &&
-                    conf.getBoolean(YarnConfiguration.YARN_ACL_ENABLE,
-                            YarnConfiguration.DEFAULT_YARN_ACL_ENABLE)) {
+        YarnConfiguration.DEFAULT_YARN_RESERVATION_ACL_ENABLE)
+        && conf.getBoolean(YarnConfiguration.YARN_ACL_ENABLE,
+            YarnConfiguration.DEFAULT_YARN_ACL_ENABLE)) {
       reservationsACLsManager = new ReservationsACLsManager(scheduler, conf);
     }
   }
 
   private void loadPlan(String planName,
       Map<ReservationId, ReservationAllocationStateProto> reservations)
-          throws PlanningException {
+      throws PlanningException {
     Plan plan = plans.get(planName);
     Resource minAllocation = getMinAllocation();
     ResourceCalculator rescCalculator = getResourceCalculator();
@@ -248,8 +255,8 @@ public abstract class AbstractReservationSystem extends AbstractService
       Class<?> planFollowerPolicyClazz =
           conf.getClassByName(planFollowerPolicyClassName);
       if (PlanFollower.class.isAssignableFrom(planFollowerPolicyClazz)) {
-        return (PlanFollower) ReflectionUtils.newInstance(
-            planFollowerPolicyClazz, conf);
+        return (PlanFollower) ReflectionUtils
+            .newInstance(planFollowerPolicyClazz, conf);
       } else {
         throw new YarnRuntimeException("Class: " + planFollowerPolicyClassName
             + " not instance of " + PlanFollower.class.getCanonicalName());
@@ -257,7 +264,8 @@ public abstract class AbstractReservationSystem extends AbstractService
     } catch (ClassNotFoundException e) {
       throw new YarnRuntimeException(
           "Could not instantiate PlanFollowerPolicy: "
-              + planFollowerPolicyClassName, e);
+              + planFollowerPolicyClassName,
+          e);
     }
   }
 
@@ -371,9 +379,8 @@ public abstract class AbstractReservationSystem extends AbstractService
   public ReservationId getNewReservationId() {
     writeLock.lock();
     try {
-      ReservationId resId =
-          ReservationId.newInstance(ResourceManager.getClusterTimeStamp(),
-              resCounter.incrementAndGet());
+      ReservationId resId = ReservationId.newInstance(
+          ResourceManager.getClusterTimeStamp(), resCounter.incrementAndGet());
       LOG.info("Allocated new reservationId: " + resId);
       return resId;
     } finally {
@@ -390,8 +397,11 @@ public abstract class AbstractReservationSystem extends AbstractService
    * Get the default reservation system corresponding to the scheduler
    * 
    * @param scheduler the scheduler for which the reservation system is required
+   *
+   * @return the {@link ReservationSystem} based on the configured scheduler
    */
-  public static String getDefaultReservationSystem(ResourceScheduler scheduler) {
+  public static String getDefaultReservationSystem(
+      ResourceScheduler scheduler) {
     if (scheduler instanceof CapacityScheduler) {
       return CapacityReservationSystem.class.getName();
     } else if (scheduler instanceof FairScheduler) {
@@ -409,12 +419,11 @@ public abstract class AbstractReservationSystem extends AbstractService
     Resource maxAllocation = getMaxAllocation();
     ResourceCalculator rescCalc = getResourceCalculator();
     Resource totCap = getPlanQueueCapacity(planQueueName);
-    Plan plan =
-        new InMemoryPlan(getRootQueueMetrics(), adPolicy,
-            getAgent(planQueuePath), totCap, planStepSize, rescCalc,
-            minAllocation, maxAllocation, planQueueName,
-            getReplanner(planQueuePath), getReservationSchedulerConfiguration()
-            .getMoveOnExpiry(planQueuePath), rmContext);
+    Plan plan = new InMemoryPlan(getRootQueueMetrics(), adPolicy,
+        getAgent(planQueuePath), totCap, planStepSize, rescCalc, minAllocation,
+        maxAllocation, planQueueName, getReplanner(planQueuePath),
+        getReservationSchedulerConfiguration().getMoveOnExpiry(planQueuePath),
+        maxPeriodicity, rmContext);
     LOG.info("Initialized plan {} based on reservable queue {}",
         plan.toString(), planQueueName);
     return plan;
@@ -477,8 +486,8 @@ public abstract class AbstractReservationSystem extends AbstractService
       Class<?> admissionPolicyClazz =
           conf.getClassByName(admissionPolicyClassName);
       if (SharingPolicy.class.isAssignableFrom(admissionPolicyClazz)) {
-        return (SharingPolicy) ReflectionUtils.newInstance(
-            admissionPolicyClazz, conf);
+        return (SharingPolicy) ReflectionUtils.newInstance(admissionPolicyClazz,
+            conf);
       } else {
         throw new YarnRuntimeException("Class: " + admissionPolicyClassName
             + " not instance of " + SharingPolicy.class.getCanonicalName());
@@ -493,8 +502,7 @@ public abstract class AbstractReservationSystem extends AbstractService
     return this.reservationsACLsManager;
   }
 
-  protected abstract ReservationSchedulerConfiguration
-      getReservationSchedulerConfiguration();
+  protected abstract ReservationSchedulerConfiguration getReservationSchedulerConfiguration();
 
   protected abstract String getPlanQueuePath(String planQueueName);
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b6e7d136/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/AbstractSchedulerPlanFollower.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/reservation/AbstractSchedulerPlanFollower.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/AbstractSchedulerPlanFollower.java
index 90357e3..9b6a0b0 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/AbstractSchedulerPlanFollower.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/AbstractSchedulerPlanFollower.java
@@ -18,6 +18,14 @@
 
 package org.apache.hadoop.yarn.server.resourcemanager.reservation;
 
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ReservationId;
 import org.apache.hadoop.yarn.api.records.Resource;
@@ -33,24 +41,17 @@ import org.apache.hadoop.yarn.util.resource.Resources;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.Comparator;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Set;
-
 public abstract class AbstractSchedulerPlanFollower implements PlanFollower {
-  private static final Logger LOG = LoggerFactory
-      .getLogger(AbstractSchedulerPlanFollower.class);
+  private static final Logger LOG =
+      LoggerFactory.getLogger(AbstractSchedulerPlanFollower.class);
 
   protected Collection<Plan> plans = new ArrayList<Plan>();
   protected YarnScheduler scheduler;
   protected Clock clock;
 
   @Override
-  public void init(Clock clock, ResourceScheduler sched, Collection<Plan> plans) {
+  public void init(Clock clock, ResourceScheduler sched,
+      Collection<Plan> plans) {
     this.clock = clock;
     this.scheduler = sched;
     this.plans.addAll(plans);
@@ -71,7 +72,7 @@ public abstract class AbstractSchedulerPlanFollower implements PlanFollower {
 
   @Override
   public synchronized void synchronizePlan(Plan plan, boolean shouldReplan) {
-     String planQueueName = plan.getQueueName();
+    String planQueueName = plan.getQueueName();
     if (LOG.isDebugEnabled()) {
       LOG.debug("Running plan follower edit policy for plan: " + planQueueName);
     }
@@ -82,12 +83,14 @@ public abstract class AbstractSchedulerPlanFollower implements PlanFollower {
       now += step - (now % step);
     }
     Queue planQueue = getPlanQueue(planQueueName);
-    if (planQueue == null) return;
+    if (planQueue == null) {
+      return;
+    }
 
     // first we publish to the plan the current availability of resources
     Resource clusterResources = scheduler.getClusterResource();
-    Resource planResources = getPlanResources(plan, planQueue,
-        clusterResources);
+    Resource planResources =
+        getPlanResources(plan, planQueue, clusterResources);
     Set<ReservationAllocation> currentReservations =
         plan.getReservationsAtTime(now);
     Set<String> curReservationNames = new HashSet<String>();
@@ -95,12 +98,11 @@ public abstract class AbstractSchedulerPlanFollower implements PlanFollower {
     int numRes = getReservedResources(now, currentReservations,
         curReservationNames, reservedResources);
     // create the default reservation queue if it doesnt exist
-    String defReservationId = getReservationIdFromQueueName(planQueueName) +
-        ReservationConstants.DEFAULT_QUEUE_SUFFIX;
-    String defReservationQueue = getReservationQueueName(planQueueName,
-        defReservationId);
-    createDefaultReservationQueue(planQueueName, planQueue,
-        defReservationId);
+    String defReservationId = getReservationIdFromQueueName(planQueueName)
+        + ReservationConstants.DEFAULT_QUEUE_SUFFIX;
+    String defReservationQueue =
+        getReservationQueueName(planQueueName, defReservationId);
+    createDefaultReservationQueue(planQueueName, planQueue, defReservationId);
     curReservationNames.add(defReservationId);
     // if the resources dedicated to this plan has shrunk invoke replanner
     boolean shouldResize = false;
@@ -149,10 +151,8 @@ public abstract class AbstractSchedulerPlanFollower implements PlanFollower {
       // sort allocations from the one giving up the most resources, to the
       // one asking for the most avoid order-of-operation errors that
       // temporarily violate 100% capacity bound
-      List<ReservationAllocation> sortedAllocations =
-          sortByDelta(
-              new ArrayList<ReservationAllocation>(currentReservations), now,
-              plan);
+      List<ReservationAllocation> sortedAllocations = sortByDelta(
+          new ArrayList<ReservationAllocation>(currentReservations), now, plan);
       for (ReservationAllocation res : sortedAllocations) {
         String currResId = res.getReservationId().toString();
         if (curReservationNames.contains(currResId)) {
@@ -163,10 +163,9 @@ public abstract class AbstractSchedulerPlanFollower implements PlanFollower {
         if (planResources.getMemorySize() > 0
             && planResources.getVirtualCores() > 0) {
           if (shouldResize) {
-            capToAssign =
-                calculateReservationToPlanProportion(
-                    plan.getResourceCalculator(), planResources,
-                    reservedResources, capToAssign);
+            capToAssign = calculateReservationToPlanProportion(
+                plan.getResourceCalculator(), planResources, reservedResources,
+                capToAssign);
           }
           targetCapacity =
               calculateReservationToPlanRatio(plan.getResourceCalculator(),
@@ -185,7 +184,8 @@ public abstract class AbstractSchedulerPlanFollower implements PlanFollower {
           maxCapacity = targetCapacity;
         }
         try {
-          setQueueEntitlement(planQueueName, currResId, targetCapacity, maxCapacity);
+          setQueueEntitlement(planQueueName, currResId, targetCapacity,
+              maxCapacity);
         } catch (YarnException e) {
           LOG.warn("Exception while trying to size reservation for plan: {}",
               currResId, planQueueName, e);
@@ -196,9 +196,10 @@ public abstract class AbstractSchedulerPlanFollower implements PlanFollower {
     // compute the default queue capacity
     float defQCap = 1.0f - totalAssignedCapacity;
     if (LOG.isDebugEnabled()) {
-      LOG.debug("PlanFollowerEditPolicyTask: total Plan Capacity: {} "
-          + "currReservation: {} default-queue capacity: {}", planResources,
-          numRes, defQCap);
+      LOG.debug(
+          "PlanFollowerEditPolicyTask: total Plan Capacity: {} "
+              + "currReservation: {} default-queue capacity: {}",
+          planResources, numRes, defQCap);
     }
     // set the default queue to eat-up all remaining capacity
     try {
@@ -225,12 +226,11 @@ public abstract class AbstractSchedulerPlanFollower implements PlanFollower {
   }
 
   protected void setQueueEntitlement(String planQueueName, String currResId,
-      float targetCapacity,
-      float maxCapacity) throws YarnException {
-    String reservationQueueName = getReservationQueueName(planQueueName,
-        currResId);
-    scheduler.setEntitlement(reservationQueueName, new QueueEntitlement(
-        targetCapacity, maxCapacity));
+      float targetCapacity, float maxCapacity) throws YarnException {
+    String reservationQueueName =
+        getReservationQueueName(planQueueName, currResId);
+    scheduler.setEntitlement(reservationQueueName,
+        new QueueEntitlement(targetCapacity, maxCapacity));
   }
 
   // Schedulers have different ways of naming queues. See YARN-2773
@@ -244,14 +244,21 @@ public abstract class AbstractSchedulerPlanFollower implements PlanFollower {
    * Then move all apps in the set of queues to the parent plan queue's default
    * reservation queue if move is enabled. Finally cleanups the queue by killing
    * any apps (if move is disabled or move failed) and removing the queue
+   *
+   * @param planQueueName the name of {@code PlanQueue}
+   * @param shouldMove flag to indicate if any running apps should be moved or
+   *          killed
+   * @param toRemove the remnant apps to clean up
+   * @param defReservationQueue the default {@code ReservationQueue} of the
+   *          {@link Plan}
    */
-  protected void cleanupExpiredQueues(String planQueueName,
-      boolean shouldMove, Set<String> toRemove, String defReservationQueue) {
+  protected void cleanupExpiredQueues(String planQueueName, boolean shouldMove,
+      Set<String> toRemove, String defReservationQueue) {
     for (String expiredReservationId : toRemove) {
       try {
         // reduce entitlement to 0
-        String expiredReservation = getReservationQueueName(planQueueName,
-            expiredReservationId);
+        String expiredReservation =
+            getReservationQueueName(planQueueName, expiredReservationId);
         setQueueEntitlement(planQueueName, expiredReservation, 0.0f, 0.0f);
         if (shouldMove) {
           moveAppsInQueueSync(expiredReservation, defReservationQueue);
@@ -275,7 +282,7 @@ public abstract class AbstractSchedulerPlanFollower implements PlanFollower {
    * reservation queue in a synchronous fashion
    */
   private void moveAppsInQueueSync(String expiredReservation,
-                                   String defReservationQueue) {
+      String defReservationQueue) {
     List<ApplicationAttemptId> activeApps =
         scheduler.getAppsInQueue(expiredReservation);
     if (activeApps.isEmpty()) {
@@ -287,16 +294,16 @@ public abstract class AbstractSchedulerPlanFollower implements PlanFollower {
         scheduler.moveApplication(app.getApplicationId(), defReservationQueue);
       } catch (YarnException e) {
         LOG.warn(
-            "Encountered unexpected error during migration of application: {}" +
-                " from reservation: {}",
+            "Encountered unexpected error during migration of application: {}"
+                + " from reservation: {}",
             app, expiredReservation, e);
       }
     }
   }
 
-  protected int getReservedResources(long now, Set<ReservationAllocation>
-      currentReservations, Set<String> curReservationNames,
-                                     Resource reservedResources) {
+  protected int getReservedResources(long now,
+      Set<ReservationAllocation> currentReservations,
+      Set<String> curReservationNames, Resource reservedResources) {
     int numRes = 0;
     if (currentReservations != null) {
       numRes = currentReservations.size();
@@ -312,23 +319,30 @@ public abstract class AbstractSchedulerPlanFollower implements PlanFollower {
    * Sort in the order from the least new amount of resources asked (likely
    * negative) to the highest. This prevents "order-of-operation" errors related
    * to exceeding 100% capacity temporarily.
+   *
+   * @param currentReservations the currently active reservations
+   * @param now the current time
+   * @param plan the {@link Plan} that is being considered
+   *
+   * @return the sorted list of {@link ReservationAllocation}s
    */
   protected List<ReservationAllocation> sortByDelta(
       List<ReservationAllocation> currentReservations, long now, Plan plan) {
-    Collections.sort(currentReservations, new ReservationAllocationComparator(
-        now, this, plan));
+    Collections.sort(currentReservations,
+        new ReservationAllocationComparator(now, this, plan));
     return currentReservations;
   }
 
   /**
-   * Get queue associated with reservable queue named
-   * @param planQueueName Name of the reservable queue
+   * Get queue associated with reservable queue named.
+   *
+   * @param planQueueName name of the reservable queue
    * @return queue associated with the reservable queue
    */
   protected abstract Queue getPlanQueue(String planQueueName);
 
   /**
-   * Resizes reservations based on currently available resources
+   * Resizes reservations based on currently available resources.
    */
   private Resource calculateReservationToPlanProportion(
       ResourceCalculator rescCalculator, Resource availablePlanResources,
@@ -338,7 +352,7 @@ public abstract class AbstractSchedulerPlanFollower implements PlanFollower {
   }
 
   /**
-   * Calculates ratio of reservationResources to planResources
+   * Calculates ratio of reservationResources to planResources.
    */
   private float calculateReservationToPlanRatio(
       ResourceCalculator rescCalculator, Resource clusterResources,
@@ -348,7 +362,7 @@ public abstract class AbstractSchedulerPlanFollower implements PlanFollower {
   }
 
   /**
-   * Check if plan resources are less than expected reservation resources
+   * Check if plan resources are less than expected reservation resources.
    */
   private boolean arePlanResourcesLessThanReservations(
       ResourceCalculator rescCalculator, Resource clusterResources,
@@ -358,38 +372,56 @@ public abstract class AbstractSchedulerPlanFollower implements PlanFollower {
   }
 
   /**
-   * Get a list of reservation queues for this planQueue
+   * Get a list of reservation queues for this planQueue.
+   *
+   * @param planQueue the queue for the current {@link Plan}
+   *
+   * @return the queues corresponding to the reservations
    */
   protected abstract List<? extends Queue> getChildReservationQueues(
       Queue planQueue);
 
   /**
-   * Add a new reservation queue for reservation currResId for this planQueue
+   * Add a new reservation queue for reservation currResId for this planQueue.
    */
-  protected abstract void addReservationQueue(
-      String planQueueName, Queue queue, String currResId);
+  protected abstract void addReservationQueue(String planQueueName, Queue queue,
+      String currResId);
 
   /**
-   * Creates the default reservation queue for use when no reservation is
-   * used for applications submitted to this planQueue
+   * Creates the default reservation queue for use when no reservation is used
+   * for applications submitted to this planQueue.
+   *
+   * @param planQueueName name of the reservable queue
+   * @param queue the queue for the current {@link Plan}
+   * @param defReservationQueue name of the default {@code ReservationQueue}
    */
-  protected abstract void createDefaultReservationQueue(
-      String planQueueName, Queue queue, String defReservationQueue);
+  protected abstract void createDefaultReservationQueue(String planQueueName,
+      Queue queue, String defReservationQueue);
 
   /**
-   * Get plan resources for this planQueue
+   * Get plan resources for this planQueue.
+   *
+   * @param plan the current {@link Plan} being considered
+   * @param clusterResources the resources available in the cluster
+   *
+   * @return the resources allocated to the specified {@link Plan}
    */
-  protected abstract Resource getPlanResources(
-      Plan plan, Queue queue, Resource clusterResources);
+  protected abstract Resource getPlanResources(Plan plan, Queue queue,
+      Resource clusterResources);
 
   /**
    * Get reservation queue resources if it exists otherwise return null.
+   *
+   * @param plan the current {@link Plan} being considered
+   * @param reservationId the identifier of the reservation
+   *
+   * @return the resources allocated to the specified reservation
    */
   protected abstract Resource getReservationQueueResourceIfExists(Plan plan,
       ReservationId reservationId);
 
-  private static class ReservationAllocationComparator implements
-      Comparator<ReservationAllocation> {
+  private static class ReservationAllocationComparator
+      implements Comparator<ReservationAllocation> {
     AbstractSchedulerPlanFollower planFollower;
     long now;
     Plan plan;
@@ -404,14 +436,12 @@ public abstract class AbstractSchedulerPlanFollower implements PlanFollower {
     private Resource getUnallocatedReservedResources(
         ReservationAllocation reservation) {
       Resource resResource;
-      Resource reservationResource = planFollower
-          .getReservationQueueResourceIfExists
-              (plan, reservation.getReservationId());
+      Resource reservationResource =
+          planFollower.getReservationQueueResourceIfExists(plan,
+              reservation.getReservationId());
       if (reservationResource != null) {
-        resResource =
-            Resources.subtract(
-                reservation.getResourcesAtTime(now),
-                reservationResource);
+        resResource = Resources.subtract(reservation.getResourcesAtTime(now),
+            reservationResource);
       } else {
         resResource = reservation.getResourcesAtTime(now);
       }
@@ -428,4 +458,3 @@ public abstract class AbstractSchedulerPlanFollower implements PlanFollower {
     }
   }
 }
-

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b6e7d136/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/InMemoryPlan.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/reservation/InMemoryPlan.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/InMemoryPlan.java
index 783fd09..9eb1820 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/InMemoryPlan.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/InMemoryPlan.java
@@ -6,9 +6,9 @@
  * 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
- *
+ * <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.
@@ -33,9 +33,10 @@ import java.util.concurrent.locks.ReentrantReadWriteLock;
 
 import org.apache.hadoop.yarn.api.records.ReservationId;
 import org.apache.hadoop.yarn.api.records.Resource;
-import org.apache.hadoop.yarn.server.resourcemanager.reservation.RLESparseResourceAllocation.RLEOperator;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
 import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore;
+import org.apache.hadoop.yarn.server.resourcemanager.reservation.RLESparseResourceAllocation.RLEOperator;
 import org.apache.hadoop.yarn.server.resourcemanager.reservation.exceptions.PlanningException;
 import org.apache.hadoop.yarn.server.resourcemanager.reservation.planning.Planner;
 import org.apache.hadoop.yarn.server.resourcemanager.reservation.planning.ReservationAgent;
@@ -64,9 +65,14 @@ public class InMemoryPlan implements Plan {
 
   private RLESparseResourceAllocation rleSparseVector;
 
+  private PeriodicRLESparseResourceAllocation periodicRle;
+
   private Map<String, RLESparseResourceAllocation> userResourceAlloc =
       new HashMap<String, RLESparseResourceAllocation>();
 
+  private Map<String, RLESparseResourceAllocation> userPeriodicResourceAlloc =
+      new HashMap<String, RLESparseResourceAllocation>();
+
   private Map<String, RLESparseResourceAllocation> userActiveReservationCount =
       new HashMap<String, RLESparseResourceAllocation>();
 
@@ -96,15 +102,27 @@ public class InMemoryPlan implements Plan {
       String queueName, Planner replanner, boolean getMoveOnExpiry,
       RMContext rmContext) {
     this(queueMetrics, policy, agent, totalCapacity, step, resCalc, minAlloc,
-        maxAlloc, queueName, replanner, getMoveOnExpiry, rmContext,
-        new UTCClock());
+        maxAlloc, queueName, replanner, getMoveOnExpiry,
+        YarnConfiguration.DEFAULT_RM_RESERVATION_SYSTEM_MAX_PERIODICITY,
+        rmContext);
   }
 
   public InMemoryPlan(QueueMetrics queueMetrics, SharingPolicy policy,
       ReservationAgent agent, Resource totalCapacity, long step,
       ResourceCalculator resCalc, Resource minAlloc, Resource maxAlloc,
       String queueName, Planner replanner, boolean getMoveOnExpiry,
-      RMContext rmContext, Clock clock) {
+      long maxPeriodicty, RMContext rmContext) {
+    this(queueMetrics, policy, agent, totalCapacity, step, resCalc, minAlloc,
+        maxAlloc, queueName, replanner, getMoveOnExpiry, maxPeriodicty,
+        rmContext, new UTCClock());
+  }
+
+  @SuppressWarnings("checkstyle:parameternumber")
+  public InMemoryPlan(QueueMetrics queueMetrics, SharingPolicy policy,
+      ReservationAgent agent, Resource totalCapacity, long step,
+      ResourceCalculator resCalc, Resource minAlloc, Resource maxAlloc,
+      String queueName, Planner replanner, boolean getMoveOnExpiry,
+      long maxPeriodicty, RMContext rmContext, Clock clock) {
     this.queueMetrics = queueMetrics;
     this.policy = policy;
     this.agent = agent;
@@ -114,6 +132,8 @@ public class InMemoryPlan implements Plan {
     this.minAlloc = minAlloc;
     this.maxAlloc = maxAlloc;
     this.rleSparseVector = new RLESparseResourceAllocation(resCalc);
+    this.periodicRle =
+        new PeriodicRLESparseResourceAllocation(resCalc, maxPeriodicty);
     this.queueName = queueName;
     this.replanner = replanner;
     this.getMoveOnExpiry = getMoveOnExpiry;
@@ -126,6 +146,39 @@ public class InMemoryPlan implements Plan {
     return queueMetrics;
   }
 
+  private RLESparseResourceAllocation getUserRLEResourceAllocation(String user,
+      long period) {
+    RLESparseResourceAllocation resAlloc = null;
+    if (period > 0) {
+      if (userPeriodicResourceAlloc.containsKey(user)) {
+        resAlloc = userPeriodicResourceAlloc.get(user);
+      } else {
+        resAlloc = new PeriodicRLESparseResourceAllocation(resCalc,
+            periodicRle.getTimePeriod());
+        userPeriodicResourceAlloc.put(user, resAlloc);
+      }
+    } else {
+      if (userResourceAlloc.containsKey(user)) {
+        resAlloc = userResourceAlloc.get(user);
+      } else {
+        resAlloc = new RLESparseResourceAllocation(resCalc);
+        userResourceAlloc.put(user, resAlloc);
+      }
+    }
+    return resAlloc;
+  }
+
+  private void gcUserRLEResourceAllocation(String user, long period) {
+    if (period > 0) {
+      if (userPeriodicResourceAlloc.get(user).isEmpty()) {
+        userPeriodicResourceAlloc.remove(user);
+      }
+    } else {
+      if (userResourceAlloc.get(user).isEmpty()) {
+        userResourceAlloc.remove(user);
+      }
+    }
+  }
 
   private void incrementAllocation(ReservationAllocation reservation) {
     assert (readWriteLock.isWriteLockedByCurrentThread());
@@ -133,11 +186,10 @@ public class InMemoryPlan implements Plan {
         reservation.getAllocationRequests();
     // check if we have encountered the user earlier and if not add an entry
     String user = reservation.getUser();
-    RLESparseResourceAllocation resAlloc = userResourceAlloc.get(user);
-    if (resAlloc == null) {
-      resAlloc = new RLESparseResourceAllocation(resCalc);
-      userResourceAlloc.put(user, resAlloc);
-    }
+    long period = reservation.getPeriodicity();
+    RLESparseResourceAllocation resAlloc =
+        getUserRLEResourceAllocation(user, period);
+
     RLESparseResourceAllocation resCount = userActiveReservationCount.get(user);
     if (resCount == null) {
       resCount = new RLESparseResourceAllocation(resCalc);
@@ -149,14 +201,43 @@ public class InMemoryPlan implements Plan {
 
     for (Map.Entry<ReservationInterval, Resource> r : allocationRequests
         .entrySet()) {
-      resAlloc.addInterval(r.getKey(), r.getValue());
-      rleSparseVector.addInterval(r.getKey(), r.getValue());
-      if (Resources.greaterThan(resCalc, totalCapacity, r.getValue(),
-          ZERO_RESOURCE)) {
-        earliestActive = Math.min(earliestActive, r.getKey().getStartTime());
-        latestActive = Math.max(latestActive, r.getKey().getEndTime());
+
+      if (period > 0L) {
+        for (int i = 0; i < periodicRle.getTimePeriod() / period; i++) {
+
+          long rStart = r.getKey().getStartTime() + i * period;
+          long rEnd = r.getKey().getEndTime() + i * period;
+
+          // handle wrap-around
+          if (rEnd > periodicRle.getTimePeriod()) {
+            long diff = rEnd - periodicRle.getTimePeriod();
+            rEnd = periodicRle.getTimePeriod();
+            ReservationInterval newInterval = new ReservationInterval(0, diff);
+            periodicRle.addInterval(newInterval, r.getValue());
+            resAlloc.addInterval(newInterval, r.getValue());
+          }
+
+          ReservationInterval newInterval =
+              new ReservationInterval(rStart, rEnd);
+          periodicRle.addInterval(newInterval, r.getValue());
+          resAlloc.addInterval(newInterval, r.getValue());
+        }
+
+      } else {
+        rleSparseVector.addInterval(r.getKey(), r.getValue());
+        resAlloc.addInterval(r.getKey(), r.getValue());
+        if (Resources.greaterThan(resCalc, totalCapacity, r.getValue(),
+            ZERO_RESOURCE)) {
+          earliestActive = Math.min(earliestActive, r.getKey().getStartTime());
+          latestActive = Math.max(latestActive, r.getKey().getEndTime());
+        }
       }
     }
+    // periodic reservations are active from start time and good till cancelled
+    if (period > 0L) {
+      earliestActive = reservation.getStartTime();
+      latestActive = Long.MAX_VALUE;
+    }
     resCount.addInterval(new ReservationInterval(earliestActive, latestActive),
         Resource.newInstance(1, 1));
   }
@@ -166,27 +247,55 @@ public class InMemoryPlan implements Plan {
     Map<ReservationInterval, Resource> allocationRequests =
         reservation.getAllocationRequests();
     String user = reservation.getUser();
-    RLESparseResourceAllocation resAlloc = userResourceAlloc.get(user);
+    long period = reservation.getPeriodicity();
+    RLESparseResourceAllocation resAlloc =
+        getUserRLEResourceAllocation(user, period);
 
     long earliestActive = Long.MAX_VALUE;
     long latestActive = Long.MIN_VALUE;
     for (Map.Entry<ReservationInterval, Resource> r : allocationRequests
         .entrySet()) {
-      resAlloc.removeInterval(r.getKey(), r.getValue());
-      rleSparseVector.removeInterval(r.getKey(), r.getValue());
-      if (Resources.greaterThan(resCalc, totalCapacity, r.getValue(),
-          ZERO_RESOURCE)) {
-        earliestActive = Math.min(earliestActive, r.getKey().getStartTime());
-        latestActive = Math.max(latestActive, r.getKey().getEndTime());
+      if (period > 0L) {
+        for (int i = 0; i < periodicRle.getTimePeriod() / period; i++) {
+
+          long rStart = r.getKey().getStartTime() + i * period;
+          long rEnd = r.getKey().getEndTime() + i * period;
+
+          // handle wrap-around
+          if (rEnd > periodicRle.getTimePeriod()) {
+            long diff = rEnd - periodicRle.getTimePeriod();
+            rEnd = periodicRle.getTimePeriod();
+            ReservationInterval newInterval = new ReservationInterval(0, diff);
+            periodicRle.removeInterval(newInterval, r.getValue());
+            resAlloc.removeInterval(newInterval, r.getValue());
+          }
+
+          ReservationInterval newInterval =
+              new ReservationInterval(rStart, rEnd);
+          periodicRle.removeInterval(newInterval, r.getValue());
+          resAlloc.removeInterval(newInterval, r.getValue());
+        }
+      } else {
+        rleSparseVector.removeInterval(r.getKey(), r.getValue());
+        resAlloc.removeInterval(r.getKey(), r.getValue());
+        if (Resources.greaterThan(resCalc, totalCapacity, r.getValue(),
+            ZERO_RESOURCE)) {
+          earliestActive = Math.min(earliestActive, r.getKey().getStartTime());
+          latestActive = Math.max(latestActive, r.getKey().getEndTime());
+        }
       }
     }
-    if (resAlloc.isEmpty()) {
-      userResourceAlloc.remove(user);
-    }
+    gcUserRLEResourceAllocation(user, period);
 
     RLESparseResourceAllocation resCount = userActiveReservationCount.get(user);
-    resCount.removeInterval(new ReservationInterval(earliestActive,
-        latestActive), Resource.newInstance(1, 1));
+    // periodic reservations are active from start time and good till cancelled
+    if (period > 0L) {
+      earliestActive = reservation.getStartTime();
+      latestActive = Long.MAX_VALUE;
+    }
+    resCount.removeInterval(
+        new ReservationInterval(earliestActive, latestActive),
+        Resource.newInstance(1, 1));
     if (resCount.isEmpty()) {
       userActiveReservationCount.remove(user);
     }
@@ -198,9 +307,9 @@ public class InMemoryPlan implements Plan {
       if (currentReservations != null) {
         Set<ReservationAllocation> flattenedReservations =
             new TreeSet<ReservationAllocation>();
-        for (Set<InMemoryReservationAllocation> reservationEntries :
-            currentReservations.values()) {
-          flattenedReservations.addAll(reservationEntries);
+        for (Set<InMemoryReservationAllocation> res : currentReservations
+            .values()) {
+          flattenedReservations.addAll(res);
         }
         return flattenedReservations;
       } else {
@@ -218,19 +327,16 @@ public class InMemoryPlan implements Plan {
     InMemoryReservationAllocation inMemReservation =
         (InMemoryReservationAllocation) reservation;
     if (inMemReservation.getUser() == null) {
-      String errMsg =
-          "The specified Reservation with ID "
-              + inMemReservation.getReservationId()
-              + " is not mapped to any user";
+      String errMsg = "The specified Reservation with ID "
+          + inMemReservation.getReservationId() + " is not mapped to any user";
       LOG.error(errMsg);
       throw new IllegalArgumentException(errMsg);
     }
     writeLock.lock();
     try {
       if (reservationTable.containsKey(inMemReservation.getReservationId())) {
-        String errMsg =
-            "The specified Reservation with ID "
-                + inMemReservation.getReservationId() + " already exists";
+        String errMsg = "The specified Reservation with ID "
+            + inMemReservation.getReservationId() + " already exists";
         LOG.error(errMsg);
         throw new IllegalArgumentException(errMsg);
       }
@@ -246,9 +352,8 @@ public class InMemoryPlan implements Plan {
               getQueueName(), inMemReservation.getReservationId().toString());
         }
       }
-      ReservationInterval searchInterval =
-          new ReservationInterval(inMemReservation.getStartTime(),
-              inMemReservation.getEndTime());
+      ReservationInterval searchInterval = new ReservationInterval(
+          inMemReservation.getStartTime(), inMemReservation.getEndTime());
       Set<InMemoryReservationAllocation> reservations =
           currentReservations.get(searchInterval);
       if (reservations == null) {
@@ -280,9 +385,8 @@ public class InMemoryPlan implements Plan {
       ReservationId resId = reservation.getReservationId();
       ReservationAllocation currReservation = getReservationById(resId);
       if (currReservation == null) {
-        String errMsg =
-            "The specified Reservation with ID " + resId
-                + " does not exist in the plan";
+        String errMsg = "The specified Reservation with ID " + resId
+            + " does not exist in the plan";
         LOG.error(errMsg);
         throw new IllegalArgumentException(errMsg);
       }
@@ -318,9 +422,8 @@ public class InMemoryPlan implements Plan {
 
   private boolean removeReservation(ReservationAllocation reservation) {
     assert (readWriteLock.isWriteLockedByCurrentThread());
-    ReservationInterval searchInterval =
-        new ReservationInterval(reservation.getStartTime(),
-            reservation.getEndTime());
+    ReservationInterval searchInterval = new ReservationInterval(
+        reservation.getStartTime(), reservation.getEndTime());
     Set<InMemoryReservationAllocation> reservations =
         currentReservations.get(searchInterval);
     if (reservations != null) {
@@ -337,16 +440,15 @@ public class InMemoryPlan implements Plan {
         currentReservations.remove(searchInterval);
       }
     } else {
-      String errMsg =
-          "The specified Reservation with ID " + reservation.getReservationId()
-              + " does not exist in the plan";
+      String errMsg = "The specified Reservation with ID "
+          + reservation.getReservationId() + " does not exist in the plan";
       LOG.error(errMsg);
       throw new IllegalArgumentException(errMsg);
     }
     reservationTable.remove(reservation.getReservationId());
     decrementAllocation(reservation);
     LOG.info("Sucessfully deleted reservation: {} in plan.",
-            reservation.getReservationId());
+        reservation.getReservationId());
     return true;
   }
 
@@ -356,9 +458,8 @@ public class InMemoryPlan implements Plan {
     try {
       ReservationAllocation reservation = getReservationById(reservationID);
       if (reservation == null) {
-        String errMsg =
-            "The specified Reservation with ID " + reservationID
-                + " does not exist in the plan";
+        String errMsg = "The specified Reservation with ID " + reservationID
+            + " does not exist in the plan";
         LOG.error(errMsg);
         throw new IllegalArgumentException(errMsg);
       }
@@ -453,66 +554,90 @@ public class InMemoryPlan implements Plan {
       long start, long end) {
     readLock.lock();
     try {
+      // merge periodic and non-periodic allocations
       RLESparseResourceAllocation userResAlloc = userResourceAlloc.get(user);
+      RLESparseResourceAllocation userPeriodicResAlloc =
+          userPeriodicResourceAlloc.get(user);
 
+      if (userResAlloc != null && userPeriodicResAlloc != null) {
+        return RLESparseResourceAllocation.merge(resCalc, totalCapacity,
+            userResAlloc, userPeriodicResAlloc, RLEOperator.add, start, end);
+      }
       if (userResAlloc != null) {
         return userResAlloc.getRangeOverlapping(start, end);
-      } else {
-        return new RLESparseResourceAllocation(resCalc);
       }
+      if (userPeriodicResAlloc != null) {
+        return userPeriodicResAlloc.getRangeOverlapping(start, end);
+      }
+    } catch (PlanningException e) {
+      LOG.warn("Exception while trying to merge periodic"
+          + " and non-periodic user allocations: {}", e.getMessage(), e);
     } finally {
       readLock.unlock();
     }
+    return new RLESparseResourceAllocation(resCalc);
   }
 
   @Override
   public Resource getTotalCommittedResources(long t) {
     readLock.lock();
     try {
-      return rleSparseVector.getCapacityAtTime(t);
+      return Resources.add(rleSparseVector.getCapacityAtTime(t),
+          periodicRle.getCapacityAtTime(t));
     } finally {
       readLock.unlock();
     }
   }
 
   @Override
-  public Set<ReservationAllocation> getReservations(ReservationId
-                    reservationID, ReservationInterval interval) {
+  public Set<ReservationAllocation> getReservations(ReservationId reservationID,
+      ReservationInterval interval) {
     return getReservations(reservationID, interval, null);
   }
 
   @Override
-  public Set<ReservationAllocation> getReservations(ReservationId
-                    reservationID, ReservationInterval interval, String user) {
+  public Set<ReservationAllocation> getReservations(ReservationId reservationID,
+      ReservationInterval interval, String user) {
     if (reservationID != null) {
       ReservationAllocation allocation = getReservationById(reservationID);
-      if (allocation == null){
+      if (allocation == null) {
         return Collections.emptySet();
       }
       return Collections.singleton(allocation);
     }
 
-    long startTime = interval == null? 0 : interval.getStartTime();
-    long endTime = interval == null? Long.MAX_VALUE : interval.getEndTime();
+    long startTime = interval == null ? 0 : interval.getStartTime();
+    long endTime = interval == null ? Long.MAX_VALUE : interval.getEndTime();
 
     ReservationInterval searchInterval =
-            new ReservationInterval(endTime, Long.MAX_VALUE);
+        new ReservationInterval(endTime, Long.MAX_VALUE);
     readLock.lock();
     try {
-      SortedMap<ReservationInterval, Set<InMemoryReservationAllocation>>
-            reservations = currentReservations.headMap(searchInterval, true);
-      if (!reservations.isEmpty()) {
-        Set<ReservationAllocation> flattenedReservations =
-                new HashSet<>();
-        for (Set<InMemoryReservationAllocation> reservationEntries :
-                reservations.values()) {
-          for (InMemoryReservationAllocation res : reservationEntries) {
-            if (res.getEndTime() > startTime) {
-              if (user != null && !user.isEmpty()
-                      && !res.getUser().equals(user)) {
-                continue;
+      SortedMap<ReservationInterval, Set<InMemoryReservationAllocation>> res =
+          currentReservations.headMap(searchInterval, true);
+      if (!res.isEmpty()) {
+        Set<ReservationAllocation> flattenedReservations = new HashSet<>();
+        for (Set<InMemoryReservationAllocation> resEntries : res.values()) {
+          for (InMemoryReservationAllocation reservation : resEntries) {
+            // validate user
+            if (user != null && !user.isEmpty()
+                && !reservation.getUser().equals(user)) {
+              continue;
+            }
+            // handle periodic reservations
+            long period = reservation.getPeriodicity();
+            if (period > 0) {
+              long t = endTime % period;
+              // check for both contained and wrap-around reservations
+              if ((t - startTime) * (t - endTime)
+                  * (startTime - endTime) >= 0) {
+                flattenedReservations.add(reservation);
+              }
+            } else {
+              // check for non-periodic reservations
+              if (reservation.getEndTime() > startTime) {
+                flattenedReservations.add(reservation);
               }
-              flattenedReservations.add(res);
             }
           }
         }
@@ -550,36 +675,82 @@ public class InMemoryPlan implements Plan {
 
   @Override
   public RLESparseResourceAllocation getAvailableResourceOverTime(String user,
-      ReservationId oldId, long start, long end) throws PlanningException {
+      ReservationId oldId, long start, long end, long period)
+      throws PlanningException {
     readLock.lock();
     try {
-      // create RLE of totCapacity
-      TreeMap<Long, Resource> totAvailable = new TreeMap<Long, Resource>();
-      totAvailable.put(start, Resources.clone(totalCapacity));
-      RLESparseResourceAllocation totRLEAvail =
-          new RLESparseResourceAllocation(totAvailable, resCalc);
-
-      // subtract used from available
-      RLESparseResourceAllocation netAvailable;
-
-      netAvailable =
-          RLESparseResourceAllocation.merge(resCalc,
-              Resources.clone(totalCapacity), totRLEAvail, rleSparseVector,
-              RLEOperator.subtractTestNonNegative, start, end);
-
-      // add back in old reservation used resources if any
-      ReservationAllocation old = reservationTable.get(oldId);
-      if (old != null) {
-        netAvailable =
-            RLESparseResourceAllocation.merge(resCalc,
-                Resources.clone(totalCapacity), netAvailable,
-                old.getResourcesOverTime(), RLEOperator.add, start, end);
+
+      // for non-periodic return simple available resources
+      if (period == 0) {
+
+        // create RLE of totCapacity
+        TreeMap<Long, Resource> totAvailable = new TreeMap<Long, Resource>();
+        totAvailable.put(start, Resources.clone(totalCapacity));
+        RLESparseResourceAllocation totRLEAvail =
+            new RLESparseResourceAllocation(totAvailable, resCalc);
+
+        // subtract used from available
+        RLESparseResourceAllocation netAvailable;
+
+        netAvailable = RLESparseResourceAllocation.merge(resCalc,
+            Resources.clone(totalCapacity), totRLEAvail, rleSparseVector,
+            RLEOperator.subtractTestNonNegative, start, end);
+
+        // remove periodic component
+        netAvailable = RLESparseResourceAllocation.merge(resCalc,
+            Resources.clone(totalCapacity), netAvailable, periodicRle,
+            RLEOperator.subtractTestNonNegative, start, end);
+
+        // add back in old reservation used resources if any
+        ReservationAllocation old = reservationTable.get(oldId);
+        if (old != null) {
+
+          RLESparseResourceAllocation addBackPrevious =
+              old.getResourcesOverTime(start, end);
+          netAvailable = RLESparseResourceAllocation.merge(resCalc,
+              Resources.clone(totalCapacity), netAvailable, addBackPrevious,
+              RLEOperator.add, start, end);
+        }
+        // lower it if this is needed by the sharing policy
+        netAvailable = getSharingPolicy().availableResources(netAvailable, this,
+            user, oldId, start, end);
+        return netAvailable;
+      } else {
+
+        if (periodicRle.getTimePeriod() % period != 0) {
+          throw new PlanningException("The reservation periodicity (" + period
+              + ") must be" + "an exact divider of the system maxPeriod ("
+              + periodicRle.getTimePeriod() + ")");
+        }
+
+        // find the minimum resources available among all the instances that fit
+        // in the LCM
+        long numInstInLCM = periodicRle.getTimePeriod() / period;
+
+        RLESparseResourceAllocation minOverLCM =
+            getAvailableResourceOverTime(user, oldId, start, end, 0);
+        for (int i = 1; i < numInstInLCM; i++) {
+
+          long rStart = start + i * period;
+          long rEnd = end + i * period;
+
+          // recursive invocation of non-periodic range (to pick raw-info)
+          RLESparseResourceAllocation snapShot =
+              getAvailableResourceOverTime(user, oldId, rStart, rEnd, 0);
+
+          // time-align on start
+          snapShot.shift(-(i * period));
+
+          // pick the minimum amount of resources in each time interval
+          minOverLCM =
+              RLESparseResourceAllocation.merge(resCalc, getTotalCapacity(),
+                  minOverLCM, snapShot, RLEOperator.min, start, end);
+
+        }
+
+        return minOverLCM;
+
       }
-      // lower it if this is needed by the sharing policy
-      netAvailable =
-          getSharingPolicy().availableResources(netAvailable, this, user,
-              oldId, start, end);
-      return netAvailable;
     } finally {
       readLock.unlock();
     }
@@ -637,7 +808,7 @@ public class InMemoryPlan implements Plan {
   public String toCumulativeString() {
     readLock.lock();
     try {
-      return rleSparseVector.toString();
+      return rleSparseVector.toString() + "\n" + periodicRle.toString();
     } finally {
       readLock.unlock();
     }
@@ -689,11 +860,18 @@ public class InMemoryPlan implements Plan {
   }
 
   @Override
-  public RLESparseResourceAllocation getCumulativeLoadOverTime(
-      long start, long end) {
+  public RLESparseResourceAllocation getCumulativeLoadOverTime(long start,
+      long end) throws PlanningException {
     readLock.lock();
     try {
-      return rleSparseVector.getRangeOverlapping(start, end);
+
+      RLESparseResourceAllocation ret =
+          rleSparseVector.getRangeOverlapping(start, end);
+      ret = RLESparseResourceAllocation.merge(resCalc, totalCapacity, ret,
+          periodicRle.getRangeOverlapping(start, end), RLEOperator.add, start,
+          end);
+
+      return ret;
     } finally {
       readLock.unlock();
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b6e7d136/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/InMemoryReservationAllocation.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/reservation/InMemoryReservationAllocation.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/InMemoryReservationAllocation.java
index 69fd43f..00c8e44 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/InMemoryReservationAllocation.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/InMemoryReservationAllocation.java
@@ -42,6 +42,7 @@ public class InMemoryReservationAllocation implements ReservationAllocation {
   private final Map<ReservationInterval, Resource> allocationRequests;
   private boolean hasGang = false;
   private long acceptedAt = -1;
+  private long periodicity = 0;
 
   private RLESparseResourceAllocation resourcesOverTime;
 
@@ -67,9 +68,16 @@ public class InMemoryReservationAllocation implements ReservationAllocation {
     this.allocationRequests = allocations;
     this.planName = planName;
     this.hasGang = hasGang;
-    resourcesOverTime = new RLESparseResourceAllocation(calculator);
-    for (Map.Entry<ReservationInterval, Resource> r : allocations
-        .entrySet()) {
+    if (contract != null && contract.getRecurrenceExpression() != null) {
+      this.periodicity = Long.parseLong(contract.getRecurrenceExpression());
+    }
+    if (periodicity > 0) {
+      resourcesOverTime =
+          new PeriodicRLESparseResourceAllocation(calculator, periodicity);
+    } else {
+      resourcesOverTime = new RLESparseResourceAllocation(calculator);
+    }
+    for (Map.Entry<ReservationInterval, Resource> r : allocations.entrySet()) {
       resourcesOverTime.addInterval(r.getKey(), r.getValue());
     }
   }
@@ -133,17 +141,33 @@ public class InMemoryReservationAllocation implements ReservationAllocation {
   }
 
   @Override
-  public RLESparseResourceAllocation getResourcesOverTime(){
+  public RLESparseResourceAllocation getResourcesOverTime() {
     return resourcesOverTime;
   }
 
   @Override
+  public RLESparseResourceAllocation getResourcesOverTime(long start,
+      long end) {
+    return resourcesOverTime.getRangeOverlapping(start, end);
+  }
+
+  @Override
+  public long getPeriodicity() {
+    return periodicity;
+  }
+
+  @Override
+  public void setPeriodicity(long period) {
+    periodicity = period;
+  }
+
+  @Override
   public String toString() {
     StringBuilder sBuf = new StringBuilder();
     sBuf.append(getReservationId()).append(" user:").append(getUser())
         .append(" startTime: ").append(getStartTime()).append(" endTime: ")
-        .append(getEndTime()).append(" alloc:\n[")
-        .append(resourcesOverTime.toString()).append("] ");
+        .append(getEndTime()).append(" Periodiciy: ").append(periodicity)
+        .append(" alloc:\n[").append(resourcesOverTime.toString()).append("] ");
     return sBuf.toString();
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b6e7d136/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/NoOverCommitPolicy.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/reservation/NoOverCommitPolicy.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/NoOverCommitPolicy.java
index 55f1d00..49d4702 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/NoOverCommitPolicy.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/NoOverCommitPolicy.java
@@ -40,7 +40,7 @@ public class NoOverCommitPolicy implements SharingPolicy {
 
     RLESparseResourceAllocation available = plan.getAvailableResourceOverTime(
         reservation.getUser(), reservation.getReservationId(),
-        reservation.getStartTime(), reservation.getEndTime());
+        reservation.getStartTime(), reservation.getEndTime(), 0);
 
     // test the reservation does not exceed what is available
     try {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b6e7d136/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/PeriodicRLESparseResourceAllocation.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/reservation/PeriodicRLESparseResourceAllocation.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/PeriodicRLESparseResourceAllocation.java
index 8e3be8b3..7bc44f5 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/PeriodicRLESparseResourceAllocation.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/PeriodicRLESparseResourceAllocation.java
@@ -18,47 +18,94 @@
 
 package org.apache.hadoop.yarn.server.resourcemanager.reservation;
 
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.NavigableMap;
+import java.util.TreeMap;
+
 import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.util.resource.ResourceCalculator;
 import org.apache.hadoop.yarn.util.resource.Resources;
-
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import com.google.common.annotations.VisibleForTesting;
+
 /**
- * This data structure stores a periodic RLESparseResourceAllocation.
+ * This data structure stores a periodic {@link RLESparseResourceAllocation}.
  * Default period is 1 day (86400000ms).
  */
-public class PeriodicRLESparseResourceAllocation extends
-    RLESparseResourceAllocation {
+public class PeriodicRLESparseResourceAllocation
+    extends RLESparseResourceAllocation {
 
   // Log
-  private static final Logger LOG = LoggerFactory
-      .getLogger(PeriodicRLESparseResourceAllocation.class);
+  private static final Logger LOG =
+      LoggerFactory.getLogger(PeriodicRLESparseResourceAllocation.class);
 
   private long timePeriod;
 
   /**
    * Constructor.
    *
-   * @param rleVector {@link RLESparseResourceAllocation} with the run-length
-              encoded data.
+   * @param resourceCalculator {@link ResourceCalculator} the resource
+   *          calculator to use.
    * @param timePeriod Time period in milliseconds.
    */
   public PeriodicRLESparseResourceAllocation(
-      RLESparseResourceAllocation rleVector, Long timePeriod) {
-    super(rleVector.getCumulative(), rleVector.getResourceCalculator());
+      ResourceCalculator resourceCalculator, Long timePeriod) {
+    super(resourceCalculator);
     this.timePeriod = timePeriod;
   }
 
   /**
    * Constructor. Default time period set to 1 day.
    *
+   * @param resourceCalculator {@link ResourceCalculator} the resource
+   *          calculator to use..
+   */
+  public PeriodicRLESparseResourceAllocation(
+      ResourceCalculator resourceCalculator) {
+    this(resourceCalculator,
+        YarnConfiguration.DEFAULT_RM_RESERVATION_SYSTEM_MAX_PERIODICITY);
+  }
+
+  /**
+   * Constructor.
+   *
    * @param rleVector {@link RLESparseResourceAllocation} with the run-length
-              encoded data.
+   *          encoded data.
+   * @param timePeriod Time period in milliseconds.
    */
+  @VisibleForTesting
   public PeriodicRLESparseResourceAllocation(
-      RLESparseResourceAllocation rleVector) {
-    this(rleVector, 86400000L);
+      RLESparseResourceAllocation rleVector, Long timePeriod) {
+    super(rleVector.getCumulative(), rleVector.getResourceCalculator());
+    this.timePeriod = timePeriod;
+
+    // make sure the PeriodicRLE is zero-based, and handles wrap-around
+    long delta = (getEarliestStartTime() % timePeriod - getEarliestStartTime());
+    shift(delta);
+
+    List<Long> toRemove = new ArrayList<>();
+    Map<Long, Resource> toAdd = new TreeMap<>();
+
+    for (Map.Entry<Long, Resource> entry : cumulativeCapacity.entrySet()) {
+      if (entry.getKey() > timePeriod) {
+        toRemove.add(entry.getKey());
+        if (entry.getValue() != null) {
+          toAdd.put(timePeriod, entry.getValue());
+          long prev = entry.getKey() % timePeriod;
+          toAdd.put(prev, this.getCapacityAtTime(prev));
+          toAdd.put(0L, entry.getValue());
+        }
+      }
+    }
+    for (Long l : toRemove) {
+      cumulativeCapacity.remove(l);
+    }
+    cumulativeCapacity.putAll(toAdd);
   }
 
   /**
@@ -78,24 +125,25 @@ public class PeriodicRLESparseResourceAllocation extends
    * The interval may include 0, but the end time must be strictly less than
    * timePeriod.
    *
-   * @param interval {@link ReservationInterval} to which the specified
-   *          resource is to be added.
+   * @param interval {@link ReservationInterval} to which the specified resource
+   *          is to be added.
    * @param resource {@link Resource} to be added to the interval specified.
    * @return true if addition is successful, false otherwise
    */
-  public boolean addInterval(ReservationInterval interval,
-      Resource resource) {
+  public boolean addInterval(ReservationInterval interval, Resource resource) {
     long startTime = interval.getStartTime();
     long endTime = interval.getEndTime();
+
     if (startTime >= 0 && endTime > startTime && endTime <= timePeriod) {
       return super.addInterval(interval, resource);
     } else {
-      LOG.info("Cannot set capacity beyond end time: " + timePeriod);
+      LOG.info("Cannot set capacity beyond end time: " + timePeriod + " was ("
+          + interval.toString() + ")");
       return false;
     }
   }
 
-   /**
+  /**
    * Removes a resource for the specified interval.
    *
    * @param interval the {@link ReservationInterval} for which the resource is
@@ -103,14 +151,15 @@ public class PeriodicRLESparseResourceAllocation extends
    * @param resource the {@link Resource} to be removed.
    * @return true if removal is successful, false otherwise
    */
-  public boolean removeInterval(
-      ReservationInterval interval, Resource resource) {
+  public boolean removeInterval(ReservationInterval interval,
+      Resource resource) {
     long startTime = interval.getStartTime();
     long endTime = interval.getEndTime();
     // If the resource to be subtracted is less than the minimum resource in
     // the range, abort removal to avoid negative capacity.
-    if (!Resources.fitsIn(
-        resource, super.getMinimumCapacityInInterval(interval))) {
+    // TODO revesit decrementing endTime
+    if (!Resources.fitsIn(resource, getMinimumCapacityInInterval(
+        new ReservationInterval(startTime, endTime - 1)))) {
       LOG.info("Request to remove more resources than what is available");
       return false;
     }
@@ -125,17 +174,16 @@ public class PeriodicRLESparseResourceAllocation extends
   /**
    * Get maximum capacity at periodic offsets from the specified time.
    *
-   * @param tick UTC time base from which offsets are specified for finding
-   *          the maximum capacity.
-   * @param period periodic offset at which capacities are evaluted.
+   * @param tick UTC time base from which offsets are specified for finding the
+   *          maximum capacity.
+   * @param period periodic offset at which capacities are evaluated.
    * @return the maximum {@link Resource} across the specified time instants.
    * @return true if removal is successful, false otherwise
    */
   public Resource getMaximumPeriodicCapacity(long tick, long period) {
     Resource maxResource;
     if (period < timePeriod) {
-      maxResource =
-          super.getMaximumPeriodicCapacity(tick % timePeriod, period);
+      maxResource = super.getMaximumPeriodicCapacity(tick % timePeriod, period);
     } else {
       // if period is greater than the length of PeriodicRLESparseAllocation,
       // only a single value exists in this interval.
@@ -164,4 +212,30 @@ public class PeriodicRLESparseResourceAllocation extends
     return ret.toString();
   }
 
+  @Override
+  public RLESparseResourceAllocation getRangeOverlapping(long start, long end) {
+    NavigableMap<Long, Resource> unrolledMap = new TreeMap<>();
+    readLock.lock();
+    try {
+      long relativeStart = (start >= 0) ? start % timePeriod : 0;
+      NavigableMap<Long, Resource> cumulativeMap = this.getCumulative();
+      Long previous = cumulativeMap.floorKey(relativeStart);
+      previous = (previous != null) ? previous : 0;
+      for (long i = 0; i <= (end - start) / timePeriod; i++) {
+        for (Map.Entry<Long, Resource> e : cumulativeMap.entrySet()) {
+          long curKey = e.getKey() + (i * timePeriod);
+          if (curKey >= previous && (start + curKey - relativeStart) <= end) {
+            unrolledMap.put(curKey, e.getValue());
+          }
+        }
+      }
+      RLESparseResourceAllocation rle =
+          new RLESparseResourceAllocation(unrolledMap, getResourceCalculator());
+      rle.shift(start - relativeStart);
+      return rle;
+    } finally {
+      readLock.unlock();
+    }
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b6e7d136/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/PlanEdit.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/reservation/PlanEdit.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/PlanEdit.java
index 504a250..9afa324 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/PlanEdit.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/PlanEdit.java
@@ -28,54 +28,58 @@ import org.apache.hadoop.yarn.server.resourcemanager.reservation.exceptions.Plan
 public interface PlanEdit extends PlanContext, PlanView {
 
   /**
-   * Add a new {@link ReservationAllocation} to the plan
+   * Add a new {@link ReservationAllocation} to the plan.
    * 
    * @param reservation the {@link ReservationAllocation} to be added to the
    *          plan
    * @param isRecovering flag to indicate if reservation is being added as part
    *          of failover or not
    * @return true if addition is successful, false otherwise
+   * @throws PlanningException if addition is unsuccessful
    */
-  public boolean addReservation(ReservationAllocation reservation,
+  boolean addReservation(ReservationAllocation reservation,
       boolean isRecovering) throws PlanningException;
 
   /**
    * Updates an existing {@link ReservationAllocation} in the plan. This is
-   * required for re-negotiation
+   * required for re-negotiation.
    * 
    * @param reservation the {@link ReservationAllocation} to be updated the plan
    * @return true if update is successful, false otherwise
+   * @throws PlanningException if update is unsuccessful
    */
-  public boolean updateReservation(ReservationAllocation reservation)
+  boolean updateReservation(ReservationAllocation reservation)
       throws PlanningException;
 
   /**
    * Delete an existing {@link ReservationAllocation} from the plan identified
    * uniquely by its {@link ReservationId}. This will generally be used for
-   * garbage collection
+   * garbage collection.
    * 
    * @param reservationID the {@link ReservationAllocation} to be deleted from
    *          the plan identified uniquely by its {@link ReservationId}
    * @return true if delete is successful, false otherwise
+   * @throws PlanningException if deletion is unsuccessful
    */
-  public boolean deleteReservation(ReservationId reservationID)
+  boolean deleteReservation(ReservationId reservationID)
       throws PlanningException;
 
   /**
    * Method invoked to garbage collect old reservations. It cleans up expired
-   * reservations that have fallen out of the sliding archival window
+   * reservations that have fallen out of the sliding archival window.
    * 
    * @param tick the current time from which the archival window is computed
+   * @throws PlanningException if archival is unsuccessful
    */
-  public void archiveCompletedReservations(long tick) throws PlanningException;
+  void archiveCompletedReservations(long tick) throws PlanningException;
 
   /**
    * Sets the overall capacity in terms of {@link Resource} assigned to this
-   * plan
+   * plan.
    * 
    * @param capacity the overall capacity in terms of {@link Resource} assigned
    *          to this plan
    */
-  public void setTotalCapacity(Resource capacity);
+  void setTotalCapacity(Resource capacity);
 
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b6e7d136/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/PlanView.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/reservation/PlanView.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/PlanView.java
index 2767993..4035f68 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/PlanView.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/PlanView.java
@@ -17,50 +17,50 @@
  */
 package org.apache.hadoop.yarn.server.resourcemanager.reservation;
 
+import java.util.Set;
+
 import org.apache.hadoop.yarn.api.records.ReservationId;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.server.resourcemanager.reservation.exceptions.PlanningException;
 
-import java.util.Set;
-
 /**
  * This interface provides a read-only view on the allocations made in this
  * plan. This methods are used for example by {@code ReservationAgent}s to
  * determine the free resources in a certain point in time, and by
  * PlanFollowerPolicy to publish this plan to the scheduler.
  */
-public interface PlanView extends PlanContext {
+interface PlanView extends PlanContext {
 
   /**
    * Return a set of {@link ReservationAllocation} identified by the user who
    * made the reservation.
    *
    * @param reservationID the unqiue id to identify the
-   * {@link ReservationAllocation}
+   *          {@link ReservationAllocation}
    * @param interval the time interval used to retrieve the reservation
-   *                 allocations from. Only reservations with start time no
-   *                 greater than the interval end time, and end time no less
-   *                 than the interval start time will be selected.
+   *          allocations from. Only reservations with start time no greater
+   *          than the interval end time, and end time no less than the interval
+   *          start time will be selected.
    * @param user the user to retrieve the reservation allocation from.
    * @return a set of {@link ReservationAllocation} identified by the user who
-   * made the reservation
+   *         made the reservation
    */
-  Set<ReservationAllocation> getReservations(ReservationId
-                    reservationID, ReservationInterval interval, String user);
+  Set<ReservationAllocation> getReservations(ReservationId reservationID,
+      ReservationInterval interval, String user);
 
   /**
    * Return a set of {@link ReservationAllocation} identified by any user.
    *
    * @param reservationID the unqiue id to identify the
-   * {@link ReservationAllocation}
+   *          {@link ReservationAllocation}
    * @param interval the time interval used to retrieve the reservation
-   *                 allocations from. Only reservations with start time no
-   *                 greater than the interval end time, and end time no less
-   *                 than the interval start time will be selected.
+   *          allocations from. Only reservations with start time no greater
+   *          than the interval end time, and end time no less than the interval
+   *          start time will be selected.
    * @return a set of {@link ReservationAllocation} identified by any user
    */
   Set<ReservationAllocation> getReservations(ReservationId reservationID,
-                    ReservationInterval interval);
+      ReservationInterval interval);
 
   /**
    * Return a {@link ReservationAllocation} identified by its
@@ -70,7 +70,7 @@ public interface PlanView extends PlanContext {
    *          {@link ReservationAllocation}
    * @return {@link ReservationAllocation} identified by the specified id
    */
-  public ReservationAllocation getReservationById(ReservationId reservationID);
+  ReservationAllocation getReservationById(ReservationId reservationID);
 
   /**
    * Return a set of {@link ReservationAllocation} that belongs to a certain
@@ -78,11 +78,10 @@ public interface PlanView extends PlanContext {
    *
    * @param user the user being considered
    * @param t the instant in time being considered
-   * @return set of active {@link ReservationAllocation}s for this
-   *         user at this time
+   * @return set of active {@link ReservationAllocation}s for this user at this
+   *         time
    */
-  public Set<ReservationAllocation> getReservationByUserAtTime(String user,
-      long t);
+  Set<ReservationAllocation> getReservationByUserAtTime(String user, long t);
 
   /**
    * Gets all the active reservations at the specified point of time
@@ -91,14 +90,14 @@ public interface PlanView extends PlanContext {
    *          requested
    * @return set of active reservations at the specified time
    */
-  public Set<ReservationAllocation> getReservationsAtTime(long tick);
+  Set<ReservationAllocation> getReservationsAtTime(long tick);
 
   /**
    * Gets all the reservations in the plan
    * 
    * @return set of all reservations handled by this Plan
    */
-  public Set<ReservationAllocation> getAllReservations();
+  Set<ReservationAllocation> getAllReservations();
 
   /**
    * Returns the total {@link Resource} reserved for all users at the specified
@@ -126,61 +125,68 @@ public interface PlanView extends PlanContext {
    * 
    * @return the time (UTC in ms) at which the first reservation starts
    */
-  public long getEarliestStartTime();
+  long getEarliestStartTime();
 
   /**
    * Returns the time (UTC in ms) at which the last reservation terminates
    *
    * @return the time (UTC in ms) at which the last reservation terminates
    */
-  public long getLastEndTime();
+  long getLastEndTime();
 
   /**
    * This method returns the amount of resources available to a given user
    * (optionally if removing a certain reservation) over the start-end time
-   * range.
+   * range. If the request is periodic (period is non-zero) we return the
+   * minimum amount of resources available to periodic reservations (in all
+   * "period" windows within the system maxPeriod / LCM).
    *
-   * @param user
-   * @param oldId
-   * @param start
-   * @param end
+   * @param user the user being considered
+   * @param oldId the identifier of the existing reservation
+   * @param start start of the time interval.
+   * @param end end of the time interval.
+   * @param period the ms periodicty for this request (loop and pick min till
+   *          maxPeriodicity)
    * @return a view of the plan as it is available to this user
-   * @throws PlanningException
+   * @throws PlanningException if operation is unsuccessful
    */
-  public RLESparseResourceAllocation getAvailableResourceOverTime(String user,
-      ReservationId oldId, long start, long end) throws PlanningException;
+  RLESparseResourceAllocation getAvailableResourceOverTime(String user,
+      ReservationId oldId, long start, long end, long period)
+      throws PlanningException;
 
   /**
    * This method returns a RLE encoded view of the user reservation count
    * utilization between start and end time.
    *
-   * @param user
-   * @param start
-   * @param end
+   * @param user the user being considered
+   * @param start start of the time interval.
+   * @param end end of the time interval.
    * @return RLE encoded view of reservation used over time
    */
-  public RLESparseResourceAllocation getReservationCountForUserOverTime(
-      String user, long start, long end);
+  RLESparseResourceAllocation getReservationCountForUserOverTime(String user,
+      long start, long end);
 
   /**
    * This method returns a RLE encoded view of the user reservation utilization
    * between start and end time.
    *
-   * @param user
-   * @param start
-   * @param end
+   * @param user the user being considered
+   * @param start start of the time interval.
+   * @param end end of the time interval.
    * @return RLE encoded view of resources used over time
    */
-  public RLESparseResourceAllocation getConsumptionForUserOverTime(String user,
+  RLESparseResourceAllocation getConsumptionForUserOverTime(String user,
       long start, long end);
 
   /**
    * Get the cumulative load over a time interval.
    *
-   * @param start Start of the time interval.
-   * @param end End of the time interval.
+   * @param start start of the time interval.
+   * @param end end of the time interval.
    * @return RLE sparse allocation.
+   * @throws PlanningException if operation is unsuccessful
    */
-  RLESparseResourceAllocation getCumulativeLoadOverTime(long start, long end);
+  RLESparseResourceAllocation getCumulativeLoadOverTime(long start, long end)
+      throws PlanningException;
 
 }


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[10/37] hadoop git commit: HDFS-12359. Re-encryption should operate with minimum KMS ACL requirements.

Posted by ae...@apache.org.
HDFS-12359. Re-encryption should operate with minimum KMS ACL requirements.


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

Branch: refs/heads/HDFS-7240
Commit: 0ba8ff4b77db11fb68111f20fb077cffddd24f17
Parents: 792eff9
Author: Xiao Chen <xi...@apache.org>
Authored: Tue Sep 5 10:07:40 2017 -0700
Committer: Xiao Chen <xi...@apache.org>
Committed: Tue Sep 5 10:08:27 2017 -0700

----------------------------------------------------------------------
 .../server/namenode/EncryptionZoneManager.java  |   8 +-
 .../server/namenode/FSDirEncryptionZoneOp.java  | 105 ++++++++++---------
 .../hdfs/server/namenode/FSNamesystem.java      |  61 ++++++-----
 .../hdfs/server/namenode/TestReencryption.java  |   4 +-
 .../namenode/TestReencryptionWithKMS.java       |  91 ++++++++++++++++
 5 files changed, 188 insertions(+), 81 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/0ba8ff4b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EncryptionZoneManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EncryptionZoneManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EncryptionZoneManager.java
index f4cf8f2..3fcf797 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EncryptionZoneManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EncryptionZoneManager.java
@@ -587,13 +587,14 @@ public class EncryptionZoneManager {
    * Re-encrypts the given encryption zone path. If the given path is not the
    * root of an encryption zone, an exception is thrown.
    */
-  XAttr reencryptEncryptionZone(final INodesInPath zoneIIP,
+  List<XAttr> reencryptEncryptionZone(final INodesInPath zoneIIP,
       final String keyVersionName) throws IOException {
     assert dir.hasWriteLock();
     if (reencryptionHandler == null) {
       throw new IOException("No key provider configured, re-encryption "
           + "operation is rejected");
     }
+    final List<XAttr> xAttrs = Lists.newArrayListWithCapacity(1);
     final INode inode = zoneIIP.getLastINode();
     final String zoneName = zoneIIP.getPath();
     checkEncryptionZoneRoot(inode, zoneName);
@@ -603,10 +604,11 @@ public class EncryptionZoneManager {
     }
     LOG.info("Zone {}({}) is submitted for re-encryption.", zoneName,
         inode.getId());
-    XAttr ret = FSDirEncryptionZoneOp
+    final XAttr xattr = FSDirEncryptionZoneOp
         .updateReencryptionSubmitted(dir, zoneIIP, keyVersionName);
+    xAttrs.add(xattr);
     reencryptionHandler.notifyNewSubmission();
-    return ret;
+    return xAttrs;
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0ba8ff4b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirEncryptionZoneOp.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirEncryptionZoneOp.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirEncryptionZoneOp.java
index 2552cf5..ee2b0f4 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirEncryptionZoneOp.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirEncryptionZoneOp.java
@@ -19,7 +19,6 @@ package org.apache.hadoop.hdfs.server.namenode;
 
 import static org.apache.hadoop.hdfs.server.common.HdfsServerConstants.CRYPTO_XATTR_FILE_ENCRYPTION_INFO;
 
-import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.security.GeneralSecurityException;
 import java.security.PrivilegedExceptionAction;
@@ -32,8 +31,8 @@ import java.util.Map;
 import org.apache.hadoop.crypto.CipherSuite;
 import org.apache.hadoop.crypto.CryptoProtocolVersion;
 import org.apache.hadoop.crypto.key.KeyProvider;
-import org.apache.hadoop.crypto.key.KeyProvider.KeyVersion;
 import org.apache.hadoop.crypto.key.KeyProviderCryptoExtension;
+import org.apache.hadoop.crypto.key.KeyProviderCryptoExtension.CryptoExtension;
 import org.apache.hadoop.crypto.key.KeyProviderCryptoExtension.EncryptedKeyVersion;
 import org.apache.hadoop.fs.FileEncryptionInfo;
 import org.apache.hadoop.fs.FileStatus;
@@ -225,37 +224,15 @@ final class FSDirEncryptionZoneOp {
     }
   }
 
-  static void reencryptEncryptionZone(final FSDirectory fsd,
-      final String zone, final String keyVersionName,
-      final boolean logRetryCache) throws IOException {
-    final List<XAttr> xAttrs = Lists.newArrayListWithCapacity(1);
-    final FSPermissionChecker pc = fsd.getPermissionChecker();
-    fsd.writeLock();
-    try {
-      final INodesInPath iip = fsd.resolvePath(pc, zone, DirOp.WRITE);
-      final XAttr xattr = fsd.ezManager
-          .reencryptEncryptionZone(iip, keyVersionName);
-      xAttrs.add(xattr);
-    } finally {
-      fsd.writeUnlock();
-    }
-    fsd.getEditLog().logSetXAttrs(zone, xAttrs, logRetryCache);
+  static List<XAttr> reencryptEncryptionZone(final FSDirectory fsd,
+      final INodesInPath iip, final String keyVersionName) throws IOException {
+    assert keyVersionName != null;
+    return fsd.ezManager.reencryptEncryptionZone(iip, keyVersionName);
   }
 
-  static void cancelReencryptEncryptionZone(final FSDirectory fsd,
-      final String zone, final boolean logRetryCache) throws IOException {
-    final List<XAttr> xattrs;
-    final FSPermissionChecker pc = fsd.getPermissionChecker();
-    fsd.writeLock();
-    try {
-      final INodesInPath iip = fsd.resolvePath(pc, zone, DirOp.WRITE);
-      xattrs = fsd.ezManager.cancelReencryptEncryptionZone(iip);
-    } finally {
-      fsd.writeUnlock();
-    }
-    if (xattrs != null && !xattrs.isEmpty()) {
-      fsd.getEditLog().logSetXAttrs(zone, xattrs, logRetryCache);
-    }
+  static List<XAttr> cancelReencryptEncryptionZone(final FSDirectory fsd,
+      final INodesInPath iip) throws IOException {
+    return fsd.ezManager.cancelReencryptEncryptionZone(iip);
   }
 
   static BatchedListEntries<ZoneReencryptionStatus> listReencryptionStatus(
@@ -698,32 +675,58 @@ final class FSDirEncryptionZoneOp {
   }
 
   /**
-   * Get the last key version name for the given EZ. This will contact
-   * the KMS to getKeyVersions.
-   * @param zone the encryption zone
-   * @param pc the permission checker
-   * @return the last element from the list of keyVersionNames returned by KMS.
-   * @throws IOException
+   * Get the current key version name for the given EZ. This will first drain
+   * the provider's local cache, then generate a new edek.
+   * <p>
+   * The encryption key version of the newly generated edek will be used as
+   * the target key version of this re-encryption - meaning all edeks'
+   * keyVersion are compared with it, and only sent to the KMS for re-encryption
+   * when the version is different.
+   * <p>
+   * Note: KeyProvider has a getCurrentKey interface, but that is under
+   * a different ACL. HDFS should not try to operate on additional ACLs, but
+   * rather use the generate ACL it already has.
    */
-  static KeyVersion getLatestKeyVersion(final FSDirectory dir,
-      final String zone, final FSPermissionChecker pc) throws IOException {
-    final EncryptionZone ez;
+  static String getCurrentKeyVersion(final FSDirectory dir, final String zone)
+      throws IOException {
     assert dir.getProvider() != null;
+    assert !dir.hasReadLock();
+    final String keyName = FSDirEncryptionZoneOp.getKeyNameForZone(dir, zone);
+    if (keyName == null) {
+      throw new IOException(zone + " is not an encryption zone.");
+    }
+    // drain the local cache of the key provider.
+    // Do not invalidateCache on the server, since that's the responsibility
+    // when rolling the key version.
+    if (dir.getProvider() instanceof CryptoExtension) {
+      ((CryptoExtension) dir.getProvider()).drain(keyName);
+    }
+    final EncryptedKeyVersion edek;
+    try {
+      edek = dir.getProvider().generateEncryptedKey(keyName);
+    } catch (GeneralSecurityException gse) {
+      throw new IOException(gse);
+    }
+    Preconditions.checkNotNull(edek);
+    return edek.getEncryptionKeyVersionName();
+  }
+
+  /**
+   * Resolve the zone to an inode, find the encryption zone info associated with
+   * that inode, and return the key name. Does not contact the KMS.
+   */
+  static String getKeyNameForZone(final FSDirectory dir, final String zone)
+      throws IOException {
+    assert dir.getProvider() != null;
+    final INodesInPath iip;
+    final FSPermissionChecker pc = dir.getPermissionChecker();
     dir.readLock();
     try {
-      final INodesInPath iip = dir.resolvePath(pc, zone, DirOp.READ);
-      if (iip.getLastINode() == null) {
-        throw new FileNotFoundException(zone + " does not exist.");
-      }
-      dir.ezManager.checkEncryptionZoneRoot(iip.getLastINode(), iip.getPath());
-      ez = FSDirEncryptionZoneOp.getEZForPath(dir, iip);
+      iip = dir.resolvePath(pc, zone, DirOp.READ);
+      dir.ezManager.checkEncryptionZoneRoot(iip.getLastINode(), zone);
+      return dir.ezManager.getKeyName(iip);
     } finally {
       dir.readUnlock();
     }
-    // Contact KMS out of locks.
-    KeyVersion currKv = dir.getProvider().getCurrentKey(ez.getKeyName());
-    Preconditions.checkNotNull(currKv,
-        "No current key versions for key name " + ez.getKeyName());
-    return currKv;
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0ba8ff4b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
index 346f046..e5604c4 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
@@ -89,7 +89,6 @@ import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_REPLICATION_DEFAULT;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_REPLICATION_KEY;
 import static org.apache.hadoop.hdfs.server.namenode.FSDirStatAndListingOp.*;
 
-import org.apache.hadoop.crypto.key.KeyProvider.KeyVersion;
 import org.apache.hadoop.hdfs.protocol.BlocksStats;
 import org.apache.hadoop.hdfs.protocol.ECBlockGroupsStats;
 import org.apache.hadoop.hdfs.protocol.OpenFileEntry;
@@ -7105,34 +7104,46 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
       throw new IOException("No key provider configured, re-encryption "
           + "operation is rejected");
     }
-    FSPermissionChecker pc = getPermissionChecker();
-    // get keyVersionName out of the lock. This keyVersionName will be used
-    // as the target keyVersion for the entire re-encryption.
-    // This means all edek's keyVersion will be compared with this one, and
-    // kms is only contacted if the edek's keyVersion is different.
-    final KeyVersion kv =
-        FSDirEncryptionZoneOp.getLatestKeyVersion(dir, zone, pc);
-    provider.invalidateCache(kv.getName());
+    String keyVersionName = null;
+    if (action == ReencryptAction.START) {
+      // get zone's latest key version name out of the lock.
+      keyVersionName = FSDirEncryptionZoneOp.getCurrentKeyVersion(dir, zone);
+      if (keyVersionName == null) {
+        throw new IOException("Failed to get key version name for " + zone);
+      }
+    }
     writeLock();
     try {
       checkSuperuserPrivilege();
       checkOperation(OperationCategory.WRITE);
-      checkNameNodeSafeMode(
-          "NameNode in safemode, cannot " + action + " re-encryption on zone "
-              + zone);
-      switch (action) {
-      case START:
-        FSDirEncryptionZoneOp
-            .reencryptEncryptionZone(dir, zone, kv.getVersionName(),
-                logRetryCache);
-        break;
-      case CANCEL:
-        FSDirEncryptionZoneOp
-            .cancelReencryptEncryptionZone(dir, zone, logRetryCache);
-        break;
-      default:
-        throw new IOException(
-            "Re-encryption action " + action + " is not supported");
+      checkNameNodeSafeMode("NameNode in safemode, cannot " + action
+          + " re-encryption on zone " + zone);
+      final FSPermissionChecker pc = dir.getPermissionChecker();
+      List<XAttr> xattrs;
+      dir.writeLock();
+      try {
+        final INodesInPath iip = dir.resolvePath(pc, zone, DirOp.WRITE);
+        if (iip.getLastINode() == null) {
+          throw new FileNotFoundException(zone + " does not exist.");
+        }
+        switch (action) {
+        case START:
+          xattrs = FSDirEncryptionZoneOp
+              .reencryptEncryptionZone(dir, iip, keyVersionName);
+          break;
+        case CANCEL:
+          xattrs =
+              FSDirEncryptionZoneOp.cancelReencryptEncryptionZone(dir, iip);
+          break;
+        default:
+          throw new IOException(
+              "Re-encryption action " + action + " is not supported");
+        }
+      } finally {
+        dir.writeUnlock();
+      }
+      if (xattrs != null && !xattrs.isEmpty()) {
+        getEditLog().logSetXAttrs(zone, xattrs, logRetryCache);
       }
     } finally {
       writeUnlock();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0ba8ff4b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestReencryption.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestReencryption.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestReencryption.java
index 4b5be2e..5612d65 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestReencryption.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestReencryption.java
@@ -103,7 +103,7 @@ public class TestReencryption {
   private static final EnumSet<CreateEncryptionZoneFlag> NO_TRASH =
       EnumSet.of(CreateEncryptionZoneFlag.NO_TRASH);
 
-  private String getKeyProviderURI() {
+  protected String getKeyProviderURI() {
     return JavaKeyStoreProvider.SCHEME_NAME + "://file" + new Path(
         testRootDir.toString(), "test.jks").toUri();
   }
@@ -149,7 +149,7 @@ public class TestReencryption {
     GenericTestUtils.setLogLevel(ReencryptionUpdater.LOG, Level.TRACE);
   }
 
-  private void setProvider() {
+  protected void setProvider() {
     // Need to set the client's KeyProvider to the NN's for JKS,
     // else the updates do not get flushed properly
     fs.getClient()

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0ba8ff4b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestReencryptionWithKMS.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestReencryptionWithKMS.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestReencryptionWithKMS.java
new file mode 100644
index 0000000..af9c381
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestReencryptionWithKMS.java
@@ -0,0 +1,91 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.server.namenode;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.crypto.key.kms.KMSClientProvider;
+import org.apache.hadoop.crypto.key.kms.server.KMSACLs;
+import org.apache.hadoop.crypto.key.kms.server.KMSConfiguration;
+import org.apache.hadoop.crypto.key.kms.server.KMSWebApp;
+import org.apache.hadoop.crypto.key.kms.server.MiniKMS;
+import org.apache.hadoop.fs.Path;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.io.File;
+import java.io.FileWriter;
+import java.io.Writer;
+import java.util.UUID;
+
+import static org.junit.Assert.assertTrue;
+
+/**
+ * Test class for re-encryption with minikms.
+ */
+public class TestReencryptionWithKMS extends TestReencryption{
+
+  private MiniKMS miniKMS;
+  private String kmsDir;
+
+  @Override
+  protected String getKeyProviderURI() {
+    return KMSClientProvider.SCHEME_NAME + "://" +
+        miniKMS.getKMSUrl().toExternalForm().replace("://", "@");
+  }
+
+  @Before
+  public void setup() throws Exception {
+    kmsDir = "target/test-classes/" + UUID.randomUUID().toString();
+    final File dir = new File(kmsDir);
+    assertTrue(dir.mkdirs());
+    MiniKMS.Builder miniKMSBuilder = new MiniKMS.Builder();
+    miniKMS = miniKMSBuilder.setKmsConfDir(dir).build();
+    miniKMS.start();
+    super.setup();
+  }
+
+  @After
+  public void teardown() {
+    super.teardown();
+    if (miniKMS != null) {
+      miniKMS.stop();
+    }
+  }
+
+  @Override
+  protected void setProvider() {
+  }
+
+  @Test
+  public void testReencryptionKMSACLs() throws Exception {
+    final Path aclPath = new Path(kmsDir, KMSConfiguration.KMS_ACLS_XML);
+    final Configuration acl = new Configuration(false);
+    acl.addResource(aclPath);
+    // should not require any of the get ACLs.
+    acl.set(KMSACLs.Type.GET.getBlacklistConfigKey(), "*");
+    acl.set(KMSACLs.Type.GET_KEYS.getBlacklistConfigKey(), "*");
+    final File kmsAcl = new File(aclPath.toString());
+    assertTrue(kmsAcl.exists());
+    try (Writer writer = new FileWriter(kmsAcl)) {
+      acl.writeXml(writer);
+    }
+    KMSWebApp.getACLs().run();
+    testReencryptionBasic();
+  }
+}


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[16/37] hadoop git commit: HADOOP-14839. DistCp log output should contain copied and deleted files and directories. Contributed by Yiqun Lin.

Posted by ae...@apache.org.
HADOOP-14839. DistCp log output should contain copied and deleted files and directories. Contributed by Yiqun Lin.


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

Branch: refs/heads/HDFS-7240
Commit: 63720ef574df6d79a9bab0628edb915d1e3a7c30
Parents: 50506e9
Author: Xiaoyu Yao <xy...@apache.org>
Authored: Tue Sep 5 21:45:30 2017 -0700
Committer: Xiaoyu Yao <xy...@apache.org>
Committed: Tue Sep 5 23:34:55 2017 -0700

----------------------------------------------------------------------
 .../apache/hadoop/tools/DistCpConstants.java    |  1 +
 .../apache/hadoop/tools/DistCpOptionSwitch.java |  7 ++
 .../org/apache/hadoop/tools/DistCpOptions.java  | 22 +++++
 .../org/apache/hadoop/tools/OptionsParser.java  |  4 +-
 .../apache/hadoop/tools/mapred/CopyMapper.java  | 32 +++++--
 .../src/site/markdown/DistCp.md.vm              |  1 +
 .../apache/hadoop/tools/TestDistCpOptions.java  | 21 ++++-
 .../hadoop/tools/mapred/TestCopyMapper.java     | 99 +++++++++++++++++++-
 .../hadoop/tools/DistCp_Counter.properties      |  1 +
 9 files changed, 177 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/63720ef5/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/DistCpConstants.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/DistCpConstants.java b/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/DistCpConstants.java
index 8991e09..0bae5d5 100644
--- a/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/DistCpConstants.java
+++ b/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/DistCpConstants.java
@@ -45,6 +45,7 @@ public class DistCpConstants {
   public static final String CONF_LABEL_ATOMIC_COPY = "distcp.atomic.copy";
   public static final String CONF_LABEL_WORK_PATH = "distcp.work.path";
   public static final String CONF_LABEL_LOG_PATH = "distcp.log.path";
+  public static final String CONF_LABEL_VERBOSE_LOG = "distcp.verbose.log";
   public static final String CONF_LABEL_IGNORE_FAILURES = "distcp.ignore.failures";
   public static final String CONF_LABEL_PRESERVE_STATUS = "distcp.preserve.status";
   public static final String CONF_LABEL_PRESERVE_RAWXATTRS =

http://git-wip-us.apache.org/repos/asf/hadoop/blob/63720ef5/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/DistCpOptionSwitch.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/DistCpOptionSwitch.java b/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/DistCpOptionSwitch.java
index 016172e0..e1c824e 100644
--- a/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/DistCpOptionSwitch.java
+++ b/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/DistCpOptionSwitch.java
@@ -119,6 +119,13 @@ public enum DistCpOptionSwitch {
       new Option("log", true, "Folder on DFS where distcp execution logs are saved")),
 
   /**
+   * Log additional info (path, size) in the SKIP/COPY log.
+   */
+  VERBOSE_LOG(DistCpConstants.CONF_LABEL_VERBOSE_LOG,
+      new Option("v", false,
+          "Log additional info (path, size) in the SKIP/COPY log")),
+
+  /**
    * Copy strategy is use. This could be dynamic or uniform size etc.
    * DistCp would use an appropriate input format based on this.
    */

http://git-wip-us.apache.org/repos/asf/hadoop/blob/63720ef5/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/DistCpOptions.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/DistCpOptions.java b/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/DistCpOptions.java
index af6cb8b..ece1a94 100644
--- a/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/DistCpOptions.java
+++ b/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/DistCpOptions.java
@@ -101,6 +101,9 @@ public final class DistCpOptions {
   // content at their s1, if src is not the same as tgt.
   private final boolean useRdiff;
 
+  /** Whether to log additional info (path, size) in the SKIP/COPY log. */
+  private final boolean verboseLog;
+
   // For both -diff and -rdiff, given the example command line switches, two
   // steps are taken:
   //   1. Sync Step. This step does renaming/deletion ops in the snapshot diff,
@@ -204,6 +207,7 @@ public final class DistCpOptions {
     this.blocksPerChunk = builder.blocksPerChunk;
 
     this.copyBufferSize = builder.copyBufferSize;
+    this.verboseLog = builder.verboseLog;
   }
 
   public Path getSourceFileListing() {
@@ -323,6 +327,10 @@ public final class DistCpOptions {
     return copyBufferSize;
   }
 
+  public boolean shouldVerboseLog() {
+    return verboseLog;
+  }
+
   /**
    * Add options to configuration. These will be used in the Mapper/committer
    *
@@ -361,6 +369,8 @@ public final class DistCpOptions {
         String.valueOf(blocksPerChunk));
     DistCpOptionSwitch.addToConf(conf, DistCpOptionSwitch.COPY_BUFFER_SIZE,
         String.valueOf(copyBufferSize));
+    DistCpOptionSwitch.addToConf(conf, DistCpOptionSwitch.VERBOSE_LOG,
+        String.valueOf(verboseLog));
   }
 
   /**
@@ -396,6 +406,7 @@ public final class DistCpOptions {
         ", filtersFile='" + filtersFile + '\'' +
         ", blocksPerChunk=" + blocksPerChunk +
         ", copyBufferSize=" + copyBufferSize +
+        ", verboseLog=" + verboseLog +
         '}';
   }
 
@@ -420,6 +431,7 @@ public final class DistCpOptions {
     private boolean append = false;
     private boolean skipCRC = false;
     private boolean blocking = true;
+    private boolean verboseLog = false;
 
     private boolean useDiff = false;
     private boolean useRdiff = false;
@@ -552,6 +564,11 @@ public final class DistCpOptions {
         throw new IllegalArgumentException(
             "-diff and -rdiff are mutually exclusive");
       }
+
+      if (verboseLog && logPath == null) {
+        throw new IllegalArgumentException(
+            "-v is valid only with -log option");
+      }
     }
 
     @VisibleForTesting
@@ -685,6 +702,11 @@ public final class DistCpOptions {
               : DistCpConstants.COPY_BUFFER_SIZE_DEFAULT;
       return this;
     }
+
+    public Builder withVerboseLog(boolean newVerboseLog) {
+      this.verboseLog = newVerboseLog;
+      return this;
+    }
   }
 
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/63720ef5/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/OptionsParser.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/OptionsParser.java b/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/OptionsParser.java
index 2bfaccf..606ed32 100644
--- a/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/OptionsParser.java
+++ b/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/OptionsParser.java
@@ -111,7 +111,9 @@ public class OptionsParser {
         .withCRC(
             command.hasOption(DistCpOptionSwitch.SKIP_CRC.getSwitch()))
         .withBlocking(
-            !command.hasOption(DistCpOptionSwitch.BLOCKING.getSwitch()));
+            !command.hasOption(DistCpOptionSwitch.BLOCKING.getSwitch()))
+        .withVerboseLog(
+            command.hasOption(DistCpOptionSwitch.VERBOSE_LOG.getSwitch()));
 
     if (command.hasOption(DistCpOptionSwitch.DIFF.getSwitch())) {
       String[] snapshots = getVals(command,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/63720ef5/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/mapred/CopyMapper.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/mapred/CopyMapper.java b/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/mapred/CopyMapper.java
index d6b3ba8..faa4aa2 100644
--- a/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/mapred/CopyMapper.java
+++ b/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/mapred/CopyMapper.java
@@ -54,6 +54,7 @@ public class CopyMapper extends Mapper<Text, CopyListingFileStatus, Text, Text>
    */
   public static enum Counter {
     COPY,         // Number of files received by the mapper for copy.
+    DIR_COPY,     // Number of directories received by the mapper for copy.
     SKIP,         // Number of files skipped.
     FAIL,         // Number of files that failed to be copied.
     BYTESCOPIED,  // Number of bytes actually copied by the copy-mapper, total.
@@ -82,6 +83,7 @@ public class CopyMapper extends Mapper<Text, CopyListingFileStatus, Text, Text>
   private boolean skipCrc = false;
   private boolean overWrite = false;
   private boolean append = false;
+  private boolean verboseLog = false;
   private EnumSet<FileAttribute> preserve = EnumSet.noneOf(FileAttribute.class);
 
   private FileSystem targetFS = null;
@@ -105,6 +107,8 @@ public class CopyMapper extends Mapper<Text, CopyListingFileStatus, Text, Text>
     skipCrc = conf.getBoolean(DistCpOptionSwitch.SKIP_CRC.getConfigLabel(), false);
     overWrite = conf.getBoolean(DistCpOptionSwitch.OVERWRITE.getConfigLabel(), false);
     append = conf.getBoolean(DistCpOptionSwitch.APPEND.getConfigLabel(), false);
+    verboseLog = conf.getBoolean(
+        DistCpOptionSwitch.VERBOSE_LOG.getConfigLabel(), false);
     preserve = DistCpUtils.unpackAttributes(conf.get(DistCpOptionSwitch.
         PRESERVE_STATUS.getConfigLabel()));
 
@@ -196,6 +200,13 @@ public class CopyMapper extends Mapper<Text, CopyListingFileStatus, Text, Text>
         updateSkipCounters(context, sourceCurrStatus);
         context.write(null, new Text("SKIP: " + sourceCurrStatus.getPath()));
 
+        if (verboseLog) {
+          context.write(null,
+              new Text("FILE_SKIPPED: source=" + sourceFileStatus.getPath()
+              + ", size=" + sourceFileStatus.getLen() + " --> "
+              + "target=" + target + ", size=" + (targetStatus == null ?
+                  0 : targetStatus.getLen())));
+        }
       } else {
         if (sourceCurrStatus.isSplit()) {
           tmpTarget = DistCpUtils.getSplitChunkPath(target, sourceCurrStatus);
@@ -203,8 +214,8 @@ public class CopyMapper extends Mapper<Text, CopyListingFileStatus, Text, Text>
         if (LOG.isDebugEnabled()) {
           LOG.debug("copying " + sourceCurrStatus + " " + tmpTarget);
         }
-        copyFileWithRetry(description, sourceCurrStatus, tmpTarget, context,
-            action, fileAttributes);
+        copyFileWithRetry(description, sourceCurrStatus, tmpTarget,
+            targetStatus, context, action, fileAttributes);
       }
       DistCpUtils.preserve(target.getFileSystem(conf), tmpTarget,
           sourceCurrStatus, fileAttributes, preserveRawXattrs);
@@ -235,9 +246,10 @@ public class CopyMapper extends Mapper<Text, CopyListingFileStatus, Text, Text>
   }
 
   private void copyFileWithRetry(String description,
-      CopyListingFileStatus sourceFileStatus, Path target, Context context,
-      FileAction action, EnumSet<DistCpOptions.FileAttribute> fileAttributes)
-      throws IOException {
+      CopyListingFileStatus sourceFileStatus, Path target,
+      FileStatus targrtFileStatus, Context context, FileAction action,
+      EnumSet<DistCpOptions.FileAttribute> fileAttributes)
+      throws IOException, InterruptedException {
     long bytesCopied;
     try {
       bytesCopied = (Long) new RetriableFileCopyCommand(skipCrc, description,
@@ -251,6 +263,14 @@ public class CopyMapper extends Mapper<Text, CopyListingFileStatus, Text, Text>
     incrementCounter(context, Counter.BYTESCOPIED, bytesCopied);
     incrementCounter(context, Counter.COPY, 1);
     totalBytesCopied += bytesCopied;
+
+    if (verboseLog) {
+      context.write(null,
+          new Text("FILE_COPIED: source=" + sourceFileStatus.getPath() + ","
+          + " size=" + sourceFileStatus.getLen() + " --> "
+          + "target=" + target + ", size=" + (targrtFileStatus == null ?
+              0 : targrtFileStatus.getLen())));
+    }
   }
 
   private void createTargetDirsWithRetry(String description,
@@ -260,7 +280,7 @@ public class CopyMapper extends Mapper<Text, CopyListingFileStatus, Text, Text>
     } catch (Exception e) {
       throw new IOException("mkdir failed for " + target, e);
     }
-    incrementCounter(context, Counter.COPY, 1);
+    incrementCounter(context, Counter.DIR_COPY, 1);
   }
 
   private static void updateSkipCounters(Context context,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/63720ef5/hadoop-tools/hadoop-distcp/src/site/markdown/DistCp.md.vm
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-distcp/src/site/markdown/DistCp.md.vm b/hadoop-tools/hadoop-distcp/src/site/markdown/DistCp.md.vm
index 925b24e..2cd01e2 100644
--- a/hadoop-tools/hadoop-distcp/src/site/markdown/DistCp.md.vm
+++ b/hadoop-tools/hadoop-distcp/src/site/markdown/DistCp.md.vm
@@ -220,6 +220,7 @@ Flag              | Description                          | Notes
 `-p[rbugpcaxt]` | Preserve r: replication number b: block size u: user g: group p: permission c: checksum-type a: ACL x: XAttr t: timestamp | When `-update` is specified, status updates will **not** be synchronized unless the file sizes also differ (i.e. unless the file is re-created). If -pa is specified, DistCp preserves the permissions also because ACLs are a super-set of permissions. The option -pr is only valid if both source and target directory are not erasure coded. **Note:** If -p option's are not specified, then by default block size is preserved.
 `-i` | Ignore failures | As explained in the Appendix, this option will keep more accurate statistics about the copy than the default case. It also preserves logs from failed copies, which can be valuable for debugging. Finally, a failing map will not cause the job to fail before all splits are attempted.
 `-log <logdir>` | Write logs to \<logdir\> | DistCp keeps logs of each file it attempts to copy as map output. If a map fails, the log output will not be retained if it is re-executed.
+`-v` | Log additional info (path, size) in the SKIP/COPY log | This option can only be used with -log option.
 `-m <num_maps>` | Maximum number of simultaneous copies | Specify the number of maps to copy data. Note that more maps may not necessarily improve throughput.
 `-overwrite` | Overwrite destination | If a map fails and `-i` is not specified, all the files in the split, not only those that failed, will be recopied. As discussed in the Usage documentation, it also changes the semantics for generating destination paths, so users should use this carefully.
 `-update` | Overwrite if source and destination differ in size, blocksize, or checksum | As noted in the preceding, this is not a "sync" operation. The criteria examined are the source and destination file sizes, blocksizes, and checksums; if they differ, the source file replaces the destination file. As discussed in the Usage documentation, it also changes the semantics for generating destination paths, so users should use this carefully.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/63720ef5/hadoop-tools/hadoop-distcp/src/test/java/org/apache/hadoop/tools/TestDistCpOptions.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-distcp/src/test/java/org/apache/hadoop/tools/TestDistCpOptions.java b/hadoop-tools/hadoop-distcp/src/test/java/org/apache/hadoop/tools/TestDistCpOptions.java
index 6b59b97..dd8ec69 100644
--- a/hadoop-tools/hadoop-distcp/src/test/java/org/apache/hadoop/tools/TestDistCpOptions.java
+++ b/hadoop-tools/hadoop-distcp/src/test/java/org/apache/hadoop/tools/TestDistCpOptions.java
@@ -287,7 +287,7 @@ public class TestDistCpOptions {
         "mapBandwidth=0.0, copyStrategy='uniformsize', preserveStatus=[], " +
         "atomicWorkPath=null, logPath=null, sourceFileListing=abc, " +
         "sourcePaths=null, targetPath=xyz, filtersFile='null'," +
-        " blocksPerChunk=0, copyBufferSize=8192}";
+        " blocksPerChunk=0, copyBufferSize=8192, verboseLog=false}";
     String optionString = option.toString();
     Assert.assertEquals(val, optionString);
     Assert.assertNotSame(DistCpOptionSwitch.ATOMIC_COMMIT.toString(),
@@ -514,4 +514,23 @@ public class TestDistCpOptions {
     Assert.assertEquals(DistCpConstants.COPY_BUFFER_SIZE_DEFAULT,
         builder.build().getCopyBufferSize());
   }
+
+  @Test
+  public void testVerboseLog() {
+    final DistCpOptions.Builder builder = new DistCpOptions.Builder(
+        Collections.singletonList(new Path("hdfs://localhost:8020/source")),
+        new Path("hdfs://localhost:8020/target/"));
+    Assert.assertFalse(builder.build().shouldVerboseLog());
+
+    try {
+      builder.withVerboseLog(true).build();
+      fail("-v should fail if -log option is not specified");
+    } catch (IllegalArgumentException e) {
+      assertExceptionContains("-v is valid only with -log option", e);
+    }
+
+    final Path logPath = new Path("hdfs://localhost:8020/logs");
+    builder.withLogPath(logPath).withVerboseLog(true);
+    Assert.assertTrue(builder.build().shouldVerboseLog());
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/63720ef5/hadoop-tools/hadoop-distcp/src/test/java/org/apache/hadoop/tools/mapred/TestCopyMapper.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-distcp/src/test/java/org/apache/hadoop/tools/mapred/TestCopyMapper.java b/hadoop-tools/hadoop-distcp/src/test/java/org/apache/hadoop/tools/mapred/TestCopyMapper.java
index 866ad6e..fd998c8 100644
--- a/hadoop-tools/hadoop-distcp/src/test/java/org/apache/hadoop/tools/mapred/TestCopyMapper.java
+++ b/hadoop-tools/hadoop-distcp/src/test/java/org/apache/hadoop/tools/mapred/TestCopyMapper.java
@@ -255,7 +255,13 @@ public class TestCopyMapper {
     context.getConfiguration().setBoolean(
         DistCpOptionSwitch.APPEND.getConfigLabel(), true);
     copyMapper.setup(context);
+
+    int numFiles = 0;
     for (Path path: pathList) {
+      if (fs.getFileStatus(path).isFile()) {
+        numFiles++;
+      }
+
       copyMapper.map(new Text(DistCpUtils.getRelativePath(new Path(SOURCE_PATH), path)),
               new CopyListingFileStatus(cluster.getFileSystem().getFileStatus(
                   path)), context);
@@ -266,7 +272,7 @@ public class TestCopyMapper {
     Assert.assertEquals(nFiles * DEFAULT_FILE_SIZE * 2, stubContext
         .getReporter().getCounter(CopyMapper.Counter.BYTESCOPIED)
         .getValue());
-    Assert.assertEquals(pathList.size(), stubContext.getReporter().
+    Assert.assertEquals(numFiles, stubContext.getReporter().
         getCounter(CopyMapper.Counter.COPY).getValue());
   }
 
@@ -295,7 +301,15 @@ public class TestCopyMapper {
 
     copyMapper.setup(context);
 
-    for (Path path: pathList) {
+    int numFiles = 0;
+    int numDirs = 0;
+    for (Path path : pathList) {
+      if (fs.getFileStatus(path).isDirectory()) {
+        numDirs++;
+      } else {
+        numFiles++;
+      }
+
       copyMapper.map(
           new Text(DistCpUtils.getRelativePath(new Path(SOURCE_PATH), path)),
           new CopyListingFileStatus(fs.getFileStatus(path)), context);
@@ -303,8 +317,10 @@ public class TestCopyMapper {
 
     // Check that the maps worked.
     verifyCopy(fs, preserveChecksum);
-    Assert.assertEquals(pathList.size(), stubContext.getReporter()
+    Assert.assertEquals(numFiles, stubContext.getReporter()
         .getCounter(CopyMapper.Counter.COPY).getValue());
+    Assert.assertEquals(numDirs, stubContext.getReporter()
+        .getCounter(CopyMapper.Counter.DIR_COPY).getValue());
     if (!preserveChecksum) {
       Assert.assertEquals(nFiles * DEFAULT_FILE_SIZE, stubContext
           .getReporter().getCounter(CopyMapper.Counter.BYTESCOPIED)
@@ -1118,4 +1134,81 @@ public class TestCopyMapper {
       e.printStackTrace();
     }
   }
+
+  @Test
+  public void testVerboseLogging() throws Exception {
+    deleteState();
+    createSourceData();
+
+    FileSystem fs = cluster.getFileSystem();
+    CopyMapper copyMapper = new CopyMapper();
+    StubContext stubContext = new StubContext(getConfiguration(), null, 0);
+    Mapper<Text, CopyListingFileStatus, Text, Text>.Context context
+            = stubContext.getContext();
+    copyMapper.setup(context);
+
+    int numFiles = 0;
+    for (Path path : pathList) {
+      if (fs.getFileStatus(path).isFile()) {
+        numFiles++;
+      }
+
+      copyMapper.map(
+          new Text(DistCpUtils.getRelativePath(new Path(SOURCE_PATH), path)),
+          new CopyListingFileStatus(fs.getFileStatus(path)), context);
+    }
+
+    // Check that the maps worked.
+    Assert.assertEquals(numFiles, stubContext.getReporter()
+        .getCounter(CopyMapper.Counter.COPY).getValue());
+
+    testCopyingExistingFiles(fs, copyMapper, context);
+    // verify the verbose log
+    // we shouldn't print verbose log since this option is disabled
+    for (Text value : stubContext.getWriter().values()) {
+      Assert.assertTrue(!value.toString().startsWith("FILE_COPIED:"));
+      Assert.assertTrue(!value.toString().startsWith("FILE_SKIPPED:"));
+    }
+
+    // test with verbose logging
+    deleteState();
+    createSourceData();
+
+    stubContext = new StubContext(getConfiguration(), null, 0);
+    context = stubContext.getContext();
+    copyMapper.setup(context);
+
+    // enables verbose logging
+    context.getConfiguration().setBoolean(
+        DistCpOptionSwitch.VERBOSE_LOG.getConfigLabel(), true);
+    copyMapper.setup(context);
+
+    for (Path path : pathList) {
+      copyMapper.map(
+          new Text(DistCpUtils.getRelativePath(new Path(SOURCE_PATH), path)),
+          new CopyListingFileStatus(fs.getFileStatus(path)), context);
+    }
+
+    Assert.assertEquals(numFiles, stubContext.getReporter()
+        .getCounter(CopyMapper.Counter.COPY).getValue());
+
+    // verify the verbose log of COPY log
+    int numFileCopied = 0;
+    for (Text value : stubContext.getWriter().values()) {
+      if (value.toString().startsWith("FILE_COPIED:")) {
+        numFileCopied++;
+      }
+    }
+    Assert.assertEquals(numFiles, numFileCopied);
+
+    // verify the verbose log of SKIP log
+    int numFileSkipped = 0;
+    testCopyingExistingFiles(fs, copyMapper, context);
+    for (Text value : stubContext.getWriter().values()) {
+      if (value.toString().startsWith("FILE_SKIPPED:")) {
+        numFileSkipped++;
+      }
+    }
+    Assert.assertEquals(numFiles, numFileSkipped);
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/63720ef5/hadoop-tools/hadoop-extras/src/main/java/org/apache/hadoop/tools/DistCp_Counter.properties
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-extras/src/main/java/org/apache/hadoop/tools/DistCp_Counter.properties b/hadoop-tools/hadoop-extras/src/main/java/org/apache/hadoop/tools/DistCp_Counter.properties
index 2234619..77f6c02 100644
--- a/hadoop-tools/hadoop-extras/src/main/java/org/apache/hadoop/tools/DistCp_Counter.properties
+++ b/hadoop-tools/hadoop-extras/src/main/java/org/apache/hadoop/tools/DistCp_Counter.properties
@@ -15,6 +15,7 @@
 CounterGroupName=       distcp
 
 COPY.name=              Files copied
+DIR_COPY.name=          Directories copied
 SKIP.name=              Files skipped
 FAIL.name=              Files failed
 BYTESCOPIED.name=       Bytes copied


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[07/37] hadoop git commit: YARN-7022. Improve click interaction in queue topology in new YARN UI. Contributed by Abdullah Yousufi.

Posted by ae...@apache.org.
YARN-7022. Improve click interaction in queue topology in new YARN UI. Contributed by Abdullah Yousufi.


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

Branch: refs/heads/HDFS-7240
Commit: ed162b7aa01a60488d0c95294d4503aac725bd9e
Parents: ef87d34
Author: Sunil G <su...@apache.org>
Authored: Tue Sep 5 08:06:43 2017 +0530
Committer: Sunil G <su...@apache.org>
Committed: Tue Sep 5 08:06:43 2017 +0530

----------------------------------------------------------------------
 .../src/main/webapp/app/components/tree-selector.js              | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/ed162b7a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/components/tree-selector.js
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/components/tree-selector.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/components/tree-selector.js
index 1a81a32..6788582 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/components/tree-selector.js
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/components/tree-selector.js
@@ -129,7 +129,7 @@ export default Ember.Component.extend({
     var nodeEnter = node.enter().append("g")
       .attr("class", "node")
       .attr("transform", function() { return "translate(" + source.y0 + "," + source.x0 + ")"; })
-      .on("mouseover", function(d){
+      .on("click", function(d){
         if (d.queueData.get("name") !== this.get("selected")) {
             document.location.href = "#/yarn-queues/" + d.queueData.get("name") + "!";
         }
@@ -145,7 +145,7 @@ export default Ember.Component.extend({
         }, 100);
 
       }.bind(this))
-    .on("click", function (d) {
+    .on("dblclick", function (d) {
       document.location.href = "#/yarn-queue/" + d.queueData.get("name") + "/info";
     });
 


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[22/37] hadoop git commit: Revert "Plan/ResourceAllocation data structure enhancements required to support recurring reservations in ReservationSystem."

Posted by ae...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/e3345e98/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestInMemoryPlan.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/reservation/TestInMemoryPlan.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestInMemoryPlan.java
index 03569d4..bc98e2f 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestInMemoryPlan.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestInMemoryPlan.java
@@ -40,7 +40,6 @@ import org.apache.hadoop.yarn.server.resourcemanager.reservation.planning.Planne
 import org.apache.hadoop.yarn.server.resourcemanager.reservation.planning.ReservationAgent;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueMetrics;
 import org.apache.hadoop.yarn.util.Clock;
-import org.apache.hadoop.yarn.util.UTCClock;
 import org.apache.hadoop.yarn.util.resource.DefaultResourceCalculator;
 import org.apache.hadoop.yarn.util.resource.ResourceCalculator;
 import org.junit.After;
@@ -48,10 +47,6 @@ import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
 
-/**
- * Testing the class {@link InMemoryPlan}.
- */
-@SuppressWarnings("checkstyle:nowhitespaceafter")
 public class TestInMemoryPlan {
 
   private String user = "yarn";
@@ -67,7 +62,6 @@ public class TestInMemoryPlan {
   private ReservationAgent agent;
   private Planner replanner;
   private RMContext context;
-  private long maxPeriodicity;
 
   @Before
   public void setUp() throws PlanningException {
@@ -78,7 +72,7 @@ public class TestInMemoryPlan {
 
     clock = mock(Clock.class);
     queueMetrics = mock(QueueMetrics.class);
-    policy = new NoOverCommitPolicy();
+    policy = mock(SharingPolicy.class);
     replanner = mock(Planner.class);
 
     when(clock.getTime()).thenReturn(1L);
@@ -101,41 +95,15 @@ public class TestInMemoryPlan {
 
   @Test
   public void testAddReservation() {
-    Plan plan = new InMemoryPlan(queueMetrics, policy, agent, totalCapacity, 1L,
-        resCalc, minAlloc, maxAlloc, planName, replanner, true, context);
+    Plan plan =
+        new InMemoryPlan(queueMetrics, policy, agent, totalCapacity, 1L,
+            resCalc, minAlloc, maxAlloc, planName, replanner, true, context);
     ReservationId reservationID =
-        ReservationSystemTestUtil.getNewReservationId();
+            ReservationSystemTestUtil.getNewReservationId();
     int[] alloc = { 10, 10, 10, 10, 10, 10 };
     int start = 100;
-    ReservationAllocation rAllocation =
-        createReservationAllocation(reservationID, start, alloc);
-    Assert.assertNull(plan.getReservationById(reservationID));
-    try {
-      plan.addReservation(rAllocation, false);
-    } catch (PlanningException e) {
-      Assert.fail(e.getMessage());
-    }
-    doAssertions(plan, rAllocation);
-    checkAllocation(plan, alloc, start, 0);
-  }
-
-  @Test
-  public void testAddPeriodicReservation() throws PlanningException {
-
-    maxPeriodicity = 100;
-    Plan plan = new InMemoryPlan(queueMetrics, policy, agent, totalCapacity, 1L,
-        resCalc, minAlloc, maxAlloc, planName, replanner, true, maxPeriodicity,
-        context, new UTCClock());
-
-    ReservationId reservationID =
-        ReservationSystemTestUtil.getNewReservationId();
-    int[] alloc = { 10, 50 };
-    int start = 10;
-    long period = 20;
-    ReservationAllocation rAllocation = createReservationAllocation(
-        reservationID, start, alloc, String.valueOf(period));
-    // use periodicity of 1hr
-    rAllocation.setPeriodicity(period);
+    ReservationAllocation rAllocation = createReservationAllocation
+            (reservationID, start, alloc);
     Assert.assertNull(plan.getReservationById(reservationID));
     try {
       plan.addReservation(rAllocation, false);
@@ -143,54 +111,32 @@ public class TestInMemoryPlan {
       Assert.fail(e.getMessage());
     }
     doAssertions(plan, rAllocation);
-    checkAllocation(plan, alloc, start, period);
-
-    RLESparseResourceAllocation available =
-        plan.getAvailableResourceOverTime(user, reservationID, 150, 330, 50);
-    System.out.println(available);
+    checkAllocation(plan, alloc, start);
   }
 
-  private void checkAllocation(Plan plan, int[] alloc, int start,
-      long periodicity) {
-    long end = start + alloc.length;
-    if (periodicity > 0) {
-      end = end + maxPeriodicity;
-    }
+  private void checkAllocation(Plan plan, int[] alloc, int start) {
     RLESparseResourceAllocation userCons =
-        plan.getConsumptionForUserOverTime(user, start, end * 3);
+        plan.getConsumptionForUserOverTime(user, start, start + alloc.length);
 
     for (int i = 0; i < alloc.length; i++) {
-      // only one instance for non-periodic reservation
-      if (periodicity <= 0) {
-        Assert.assertEquals(Resource.newInstance(1024 * (alloc[i]), (alloc[i])),
-            plan.getTotalCommittedResources(start + i));
-        Assert.assertEquals(Resource.newInstance(1024 * (alloc[i]), (alloc[i])),
-            userCons.getCapacityAtTime(start + i));
-      } else {
-        // periodic reservations should repeat
-        long y = 0;
-        Resource res = Resource.newInstance(1024 * (alloc[i]), (alloc[i]));
-        while (y <= end * 2) {
-          Assert.assertEquals("At time: " + start + i + y, res,
-              plan.getTotalCommittedResources(start + i + y));
-          Assert.assertEquals(" At time: " + (start + i + y), res,
-              userCons.getCapacityAtTime(start + i + y));
-          y = y + periodicity;
-        }
-      }
+      Assert.assertEquals(Resource.newInstance(1024 * (alloc[i]), (alloc[i])),
+          plan.getTotalCommittedResources(start + i));
+      Assert.assertEquals(Resource.newInstance(1024 * (alloc[i]), (alloc[i])),
+          userCons.getCapacityAtTime(start + i));
     }
   }
 
   @Test
   public void testAddEmptyReservation() {
-    Plan plan = new InMemoryPlan(queueMetrics, policy, agent, totalCapacity, 1L,
-        resCalc, minAlloc, maxAlloc, planName, replanner, true, context);
+    Plan plan =
+        new InMemoryPlan(queueMetrics, policy, agent, totalCapacity, 1L,
+            resCalc, minAlloc, maxAlloc, planName, replanner, true, context);
     ReservationId reservationID =
-        ReservationSystemTestUtil.getNewReservationId();
+            ReservationSystemTestUtil.getNewReservationId();
     int[] alloc = {};
     int start = 100;
-    ReservationAllocation rAllocation =
-        createReservationAllocation(reservationID, start, alloc);
+    ReservationAllocation rAllocation = createReservationAllocation
+            (reservationID, start, alloc);
     Assert.assertNull(plan.getReservationById(reservationID));
     try {
       plan.addReservation(rAllocation, false);
@@ -202,14 +148,15 @@ public class TestInMemoryPlan {
   @Test
   public void testAddReservationAlreadyExists() {
     // First add a reservation
-    Plan plan = new InMemoryPlan(queueMetrics, policy, agent, totalCapacity, 1L,
-        resCalc, minAlloc, maxAlloc, planName, replanner, true, context);
+    Plan plan =
+        new InMemoryPlan(queueMetrics, policy, agent, totalCapacity, 1L,
+            resCalc, minAlloc, maxAlloc, planName, replanner, true, context);
     ReservationId reservationID =
-        ReservationSystemTestUtil.getNewReservationId();
+            ReservationSystemTestUtil.getNewReservationId();
     int[] alloc = { 10, 10, 10, 10, 10, 10 };
     int start = 100;
-    ReservationAllocation rAllocation =
-        createReservationAllocation(reservationID, start, alloc);
+    ReservationAllocation rAllocation = createReservationAllocation
+            (reservationID, start, alloc);
     Assert.assertNull(plan.getReservationById(reservationID));
     try {
       plan.addReservation(rAllocation, false);
@@ -217,7 +164,7 @@ public class TestInMemoryPlan {
       Assert.fail(e.getMessage());
     }
     doAssertions(plan, rAllocation);
-    checkAllocation(plan, alloc, start, 0);
+    checkAllocation(plan, alloc, start);
 
     // Try to add it again
     try {
@@ -233,15 +180,16 @@ public class TestInMemoryPlan {
 
   @Test
   public void testUpdateReservation() {
-    Plan plan = new InMemoryPlan(queueMetrics, policy, agent, totalCapacity, 1L,
-        resCalc, minAlloc, maxAlloc, planName, replanner, true, context);
+    Plan plan =
+        new InMemoryPlan(queueMetrics, policy, agent, totalCapacity, 1L,
+            resCalc, minAlloc, maxAlloc, planName, replanner, true, context);
     ReservationId reservationID =
         ReservationSystemTestUtil.getNewReservationId();
     // First add a reservation
     int[] alloc = { 10, 10, 10, 10, 10, 10 };
     int start = 100;
-    ReservationAllocation rAllocation =
-        createReservationAllocation(reservationID, start, alloc);
+    ReservationAllocation rAllocation = createReservationAllocation
+            (reservationID, start, alloc);
     Assert.assertNull(plan.getReservationById(reservationID));
     try {
       plan.addReservation(rAllocation, false);
@@ -262,8 +210,8 @@ public class TestInMemoryPlan {
     // Now update it
     start = 110;
     int[] updatedAlloc = { 0, 5, 10, 10, 5, 0 };
-    rAllocation =
-        createReservationAllocation(reservationID, start, updatedAlloc, true);
+    rAllocation = createReservationAllocation(reservationID, start,
+            updatedAlloc, true);
     try {
       plan.updateReservation(rAllocation);
     } catch (PlanningException e) {
@@ -271,71 +219,32 @@ public class TestInMemoryPlan {
     }
     doAssertions(plan, rAllocation);
 
-    userCons = plan.getConsumptionForUserOverTime(user, start,
-        start + updatedAlloc.length);
+    userCons =
+        plan.getConsumptionForUserOverTime(user, start, start
+            + updatedAlloc.length);
 
     for (int i = 0; i < updatedAlloc.length; i++) {
-      Assert.assertEquals(Resource.newInstance(1024 * (updatedAlloc[i] + i),
-          updatedAlloc[i] + i), plan.getTotalCommittedResources(start + i));
-      Assert.assertEquals(Resource.newInstance(1024 * (updatedAlloc[i] + i),
-          updatedAlloc[i] + i), userCons.getCapacityAtTime(start + i));
-    }
-  }
-
-  @Test
-  public void testUpdatePeriodicReservation() {
-    Plan plan = new InMemoryPlan(queueMetrics, policy, agent, totalCapacity, 1L,
-        resCalc, minAlloc, maxAlloc, planName, replanner, true, context);
-    // First add a reservation
-    ReservationId reservationID =
-        ReservationSystemTestUtil.getNewReservationId();
-    int[] alloc = { 10, 20 };
-    int start = 100;
-    ReservationAllocation rAllocation =
-        createReservationAllocation(reservationID, start, alloc);
-    // use periodicity of 1hr
-    long period = 3600000;
-    rAllocation.getReservationDefinition()
-        .setRecurrenceExpression(String.valueOf(period));
-    rAllocation.setPeriodicity(period);
-    Assert.assertNull(plan.getReservationById(reservationID));
-    try {
-      plan.addReservation(rAllocation, false);
-    } catch (PlanningException e) {
-      Assert.fail(e.getMessage());
-    }
-    System.out.println(plan.toString());
-    doAssertions(plan, rAllocation);
-    checkAllocation(plan, alloc, start, period);
-
-    // Now update it
-    start = 110;
-    int[] updatedAlloc = { 30, 40, 50 };
-    rAllocation =
-        createReservationAllocation(reservationID, start, updatedAlloc);
-    rAllocation.getReservationDefinition()
-        .setRecurrenceExpression(String.valueOf(period));
-    rAllocation.setPeriodicity(period);
-    try {
-      plan.updateReservation(rAllocation);
-    } catch (PlanningException e) {
-      Assert.fail(e.getMessage());
+      Assert.assertEquals(
+     Resource.newInstance(1024 * (updatedAlloc[i] + i), updatedAlloc[i]
+              + i), plan.getTotalCommittedResources(start + i));
+      Assert.assertEquals(
+          Resource.newInstance(1024 * (updatedAlloc[i] + i), updatedAlloc[i]
+              + i), userCons.getCapacityAtTime(start + i));
     }
-    doAssertions(plan, rAllocation);
-    checkAllocation(plan, updatedAlloc, start, period);
   }
 
   @Test
   public void testUpdateNonExistingReservation() {
-    Plan plan = new InMemoryPlan(queueMetrics, policy, agent, totalCapacity, 1L,
-        resCalc, minAlloc, maxAlloc, planName, replanner, true, context);
+    Plan plan =
+        new InMemoryPlan(queueMetrics, policy, agent, totalCapacity, 1L,
+            resCalc, minAlloc, maxAlloc, planName, replanner, true, context);
     ReservationId reservationID =
         ReservationSystemTestUtil.getNewReservationId();
     // Try to update a reservation without adding
     int[] alloc = { 10, 10, 10, 10, 10, 10 };
     int start = 100;
     ReservationAllocation rAllocation =
-        createReservationAllocation(reservationID, start, alloc);
+            createReservationAllocation(reservationID, start, alloc);
     Assert.assertNull(plan.getReservationById(reservationID));
     try {
       plan.updateReservation(rAllocation);
@@ -351,14 +260,15 @@ public class TestInMemoryPlan {
   @Test
   public void testDeleteReservation() {
     // First add a reservation
-    Plan plan = new InMemoryPlan(queueMetrics, policy, agent, totalCapacity, 1L,
-        resCalc, minAlloc, maxAlloc, planName, replanner, true, context);
+    Plan plan =
+        new InMemoryPlan(queueMetrics, policy, agent, totalCapacity, 1L,
+            resCalc, minAlloc, maxAlloc, planName, replanner, true, context);
     ReservationId reservationID =
         ReservationSystemTestUtil.getNewReservationId();
     int[] alloc = { 10, 10, 10, 10, 10, 10 };
     int start = 100;
     ReservationAllocation rAllocation =
-        createReservationAllocation(reservationID, start, alloc, true);
+            createReservationAllocation(reservationID, start, alloc, true);
     Assert.assertNull(plan.getReservationById(reservationID));
     try {
       plan.addReservation(rAllocation, false);
@@ -397,46 +307,10 @@ public class TestInMemoryPlan {
   }
 
   @Test
-  public void testDeletePeriodicReservation() {
-    Plan plan = new InMemoryPlan(queueMetrics, policy, agent, totalCapacity, 1L,
-        resCalc, minAlloc, maxAlloc, planName, replanner, true, context);
-    // First add a reservation
-    ReservationId reservationID =
-        ReservationSystemTestUtil.getNewReservationId();
-    int[] alloc = { 10, 20 };
-    int start = 100;
-    ReservationAllocation rAllocation =
-        createReservationAllocation(reservationID, start, alloc);
-    // use periodicity of 1hr
-    long period = 3600000;
-    rAllocation.getReservationDefinition()
-        .setRecurrenceExpression(String.valueOf(period));
-    rAllocation.setPeriodicity(period);
-    Assert.assertNull(plan.getReservationById(reservationID));
-    try {
-      plan.addReservation(rAllocation, false);
-    } catch (PlanningException e) {
-      Assert.fail(e.getMessage());
-    }
-    System.out.println(plan.toString());
-    doAssertions(plan, rAllocation);
-    checkAllocation(plan, alloc, start, period);
-
-    // Now delete it
-    try {
-      plan.deleteReservation(reservationID);
-    } catch (PlanningException e) {
-      Assert.fail(e.getMessage());
-    }
-    Assert.assertNull(plan.getReservationById(reservationID));
-    System.out.print(plan);
-    checkAllocation(plan, new int[] { 0, 0 }, start, period);
-  }
-
-  @Test
   public void testDeleteNonExistingReservation() {
-    Plan plan = new InMemoryPlan(queueMetrics, policy, agent, totalCapacity, 1L,
-        resCalc, minAlloc, maxAlloc, planName, replanner, true, context);
+    Plan plan =
+        new InMemoryPlan(queueMetrics, policy, agent, totalCapacity, 1L,
+            resCalc, minAlloc, maxAlloc, planName, replanner, true, context);
     ReservationId reservationID =
         ReservationSystemTestUtil.getNewReservationId();
     // Try to delete a reservation without adding
@@ -454,9 +328,8 @@ public class TestInMemoryPlan {
 
   @Test
   public void testArchiveCompletedReservations() {
-    SharingPolicy sharingPolicy = mock(SharingPolicy.class);
     Plan plan =
-        new InMemoryPlan(queueMetrics, sharingPolicy, agent, totalCapacity, 1L,
+        new InMemoryPlan(queueMetrics, policy, agent, totalCapacity, 1L,
             resCalc, minAlloc, maxAlloc, planName, replanner, true, context);
     ReservationId reservationID1 =
         ReservationSystemTestUtil.getNewReservationId();
@@ -464,7 +337,7 @@ public class TestInMemoryPlan {
     int[] alloc1 = { 10, 10, 10, 10, 10, 10 };
     int start = 100;
     ReservationAllocation rAllocation =
-        createReservationAllocation(reservationID1, start, alloc1);
+            createReservationAllocation(reservationID1, start, alloc1);
     Assert.assertNull(plan.getReservationById(reservationID1));
     try {
       plan.addReservation(rAllocation, false);
@@ -472,14 +345,15 @@ public class TestInMemoryPlan {
       Assert.fail(e.getMessage());
     }
     doAssertions(plan, rAllocation);
-    checkAllocation(plan, alloc1, start, 0);
+    checkAllocation(plan, alloc1, start);
+
 
     // Now add another one
     ReservationId reservationID2 =
         ReservationSystemTestUtil.getNewReservationId();
     int[] alloc2 = { 0, 5, 10, 5, 0 };
     rAllocation =
-        createReservationAllocation(reservationID2, start, alloc2, true);
+            createReservationAllocation(reservationID2, start, alloc2, true);
     Assert.assertNull(plan.getReservationById(reservationID2));
     try {
       plan.addReservation(rAllocation, false);
@@ -493,18 +367,16 @@ public class TestInMemoryPlan {
 
     for (int i = 0; i < alloc2.length; i++) {
       Assert.assertEquals(
-          Resource.newInstance(1024 * (alloc1[i] + alloc2[i] + i),
-              alloc1[i] + alloc2[i] + i),
-          plan.getTotalCommittedResources(start + i));
+          Resource.newInstance(1024 * (alloc1[i] + alloc2[i] + i), alloc1[i]
+              + alloc2[i] + i), plan.getTotalCommittedResources(start + i));
       Assert.assertEquals(
-          Resource.newInstance(1024 * (alloc1[i] + alloc2[i] + i),
-              alloc1[i] + alloc2[i] + i),
-          userCons.getCapacityAtTime(start + i));
+          Resource.newInstance(1024 * (alloc1[i] + alloc2[i] + i), alloc1[i]
+              + alloc2[i] + i), userCons.getCapacityAtTime(start + i));
     }
 
     // Now archive completed reservations
     when(clock.getTime()).thenReturn(106L);
-    when(sharingPolicy.getValidWindow()).thenReturn(1L);
+    when(policy.getValidWindow()).thenReturn(1L);
     try {
       // will only remove 2nd reservation as only that has fallen out of the
       // archival window
@@ -514,7 +386,7 @@ public class TestInMemoryPlan {
     }
     Assert.assertNotNull(plan.getReservationById(reservationID1));
     Assert.assertNull(plan.getReservationById(reservationID2));
-    checkAllocation(plan, alloc1, start, 0);
+    checkAllocation(plan, alloc1, start);
 
     when(clock.getTime()).thenReturn(107L);
     try {
@@ -539,14 +411,15 @@ public class TestInMemoryPlan {
 
   @Test
   public void testGetReservationsById() {
-    Plan plan = new InMemoryPlan(queueMetrics, policy, agent, totalCapacity, 1L,
-        resCalc, minAlloc, maxAlloc, planName, replanner, true, context);
+    Plan plan =
+            new InMemoryPlan(queueMetrics, policy, agent, totalCapacity, 1L,
+                    resCalc, minAlloc, maxAlloc, planName, replanner, true, context);
     ReservationId reservationID =
-        ReservationSystemTestUtil.getNewReservationId();
-    int[] alloc = { 10, 10, 10, 10, 10, 10 };
+            ReservationSystemTestUtil.getNewReservationId();
+    int[] alloc = {10, 10, 10, 10, 10, 10};
     int start = 100;
-    ReservationAllocation rAllocation =
-        createReservationAllocation(reservationID, start, alloc);
+    ReservationAllocation rAllocation = createReservationAllocation
+            (reservationID, start, alloc);
     Assert.assertNull(plan.getReservationById(reservationID));
     try {
       plan.addReservation(rAllocation, false);
@@ -556,30 +429,31 @@ public class TestInMemoryPlan {
 
     // Verify that get by reservation id works.
     Set<ReservationAllocation> rAllocations =
-        plan.getReservations(reservationID, null, "");
+            plan.getReservations(reservationID, null, "");
     Assert.assertTrue(rAllocations.size() == 1);
-    Assert.assertTrue(rAllocation
-        .compareTo((ReservationAllocation) rAllocations.toArray()[0]) == 0);
+    Assert.assertTrue(rAllocation.compareTo(
+            (ReservationAllocation) rAllocations.toArray()[0]) == 0);
 
     // Verify that get by reservation id works even when time range
     // and user is invalid.
     ReservationInterval interval = new ReservationInterval(0, 0);
     rAllocations = plan.getReservations(reservationID, interval, "invalid");
     Assert.assertTrue(rAllocations.size() == 1);
-    Assert.assertTrue(rAllocation
-        .compareTo((ReservationAllocation) rAllocations.toArray()[0]) == 0);
+    Assert.assertTrue(rAllocation.compareTo(
+            (ReservationAllocation) rAllocations.toArray()[0]) == 0);
   }
 
   @Test
   public void testGetReservationsByInvalidId() {
-    Plan plan = new InMemoryPlan(queueMetrics, policy, agent, totalCapacity, 1L,
-        resCalc, minAlloc, maxAlloc, planName, replanner, true, context);
+    Plan plan =
+            new InMemoryPlan(queueMetrics, policy, agent, totalCapacity, 1L,
+                    resCalc, minAlloc, maxAlloc, planName, replanner, true, context);
     ReservationId reservationID =
-        ReservationSystemTestUtil.getNewReservationId();
-    int[] alloc = { 10, 10, 10, 10, 10, 10 };
+            ReservationSystemTestUtil.getNewReservationId();
+    int[] alloc = {10, 10, 10, 10, 10, 10};
     int start = 100;
-    ReservationAllocation rAllocation =
-        createReservationAllocation(reservationID, start, alloc);
+    ReservationAllocation rAllocation = createReservationAllocation
+            (reservationID, start, alloc);
     Assert.assertNull(plan.getReservationById(reservationID));
     try {
       plan.addReservation(rAllocation, false);
@@ -589,22 +463,23 @@ public class TestInMemoryPlan {
 
     // If reservationId is null, then nothing is returned.
     ReservationId invalidReservationID =
-        ReservationSystemTestUtil.getNewReservationId();
+            ReservationSystemTestUtil.getNewReservationId();
     Set<ReservationAllocation> rAllocations =
-        plan.getReservations(invalidReservationID, null, "");
+            plan.getReservations(invalidReservationID, null, "");
     Assert.assertTrue(rAllocations.size() == 0);
   }
 
   @Test
   public void testGetReservationsByTimeInterval() {
-    Plan plan = new InMemoryPlan(queueMetrics, policy, agent, totalCapacity, 1L,
-        resCalc, minAlloc, maxAlloc, planName, replanner, true, context);
+    Plan plan =
+            new InMemoryPlan(queueMetrics, policy, agent, totalCapacity, 1L,
+                    resCalc, minAlloc, maxAlloc, planName, replanner, true, context);
     ReservationId reservationID =
-        ReservationSystemTestUtil.getNewReservationId();
-    int[] alloc = { 10, 10, 10, 10, 10, 10 };
+            ReservationSystemTestUtil.getNewReservationId();
+    int[] alloc = {10, 10, 10, 10, 10, 10};
     int start = 100;
-    ReservationAllocation rAllocation =
-        createReservationAllocation(reservationID, start, alloc);
+    ReservationAllocation rAllocation = createReservationAllocation
+            (reservationID, start, alloc);
     Assert.assertNull(plan.getReservationById(reservationID));
     try {
       plan.addReservation(rAllocation, false);
@@ -614,24 +489,23 @@ public class TestInMemoryPlan {
 
     // Verify that get by time interval works if the selection interval
     // completely overlaps with the allocation.
-    ReservationInterval interval = new ReservationInterval(
-        rAllocation.getStartTime(), rAllocation.getEndTime());
+    ReservationInterval interval = new ReservationInterval(rAllocation
+            .getStartTime(), rAllocation.getEndTime());
     Set<ReservationAllocation> rAllocations =
-        plan.getReservations(null, interval, "");
+            plan.getReservations(null, interval, "");
     Assert.assertTrue(rAllocations.size() == 1);
-    Assert.assertTrue(rAllocation
-        .compareTo((ReservationAllocation) rAllocations.toArray()[0]) == 0);
+    Assert.assertTrue(rAllocation.compareTo(
+            (ReservationAllocation) rAllocations.toArray()[0]) == 0);
 
     // Verify that get by time interval works if the selection interval
     // falls within the allocation
     long duration = rAllocation.getEndTime() - rAllocation.getStartTime();
-    interval = new ReservationInterval(
-        rAllocation.getStartTime() + duration * (long) 0.3,
-        rAllocation.getEndTime() - duration * (long) 0.3);
+    interval = new ReservationInterval(rAllocation.getStartTime() + duration
+            * (long)0.3, rAllocation.getEndTime() - duration * (long)0.3);
     rAllocations = plan.getReservations(null, interval, "");
     Assert.assertTrue(rAllocations.size() == 1);
-    Assert.assertTrue(rAllocation
-        .compareTo((ReservationAllocation) rAllocations.toArray()[0]) == 0);
+    Assert.assertTrue(rAllocation.compareTo(
+            (ReservationAllocation) rAllocations.toArray()[0]) == 0);
 
     // Verify that get by time interval selects 1 allocation if the end
     // time of the selection interval falls right at the start of the
@@ -639,13 +513,13 @@ public class TestInMemoryPlan {
     interval = new ReservationInterval(0, rAllocation.getStartTime());
     rAllocations = plan.getReservations(null, interval, "");
     Assert.assertTrue(rAllocations.size() == 1);
-    Assert.assertTrue(rAllocation
-        .compareTo((ReservationAllocation) rAllocations.toArray()[0]) == 0);
+    Assert.assertTrue(rAllocation.compareTo(
+            (ReservationAllocation) rAllocations.toArray()[0]) == 0);
 
     // Verify that get by time interval selects no reservations if the start
     // time of the selection interval falls right at the end of the allocation.
-    interval =
-        new ReservationInterval(rAllocation.getEndTime(), Long.MAX_VALUE);
+    interval = new ReservationInterval(rAllocation
+            .getEndTime(), Long.MAX_VALUE);
     rAllocations = plan.getReservations(null, interval, "");
     Assert.assertTrue(rAllocations.size() == 0);
 
@@ -658,14 +532,15 @@ public class TestInMemoryPlan {
 
   @Test
   public void testGetReservationsAtTime() {
-    Plan plan = new InMemoryPlan(queueMetrics, policy, agent, totalCapacity, 1L,
-        resCalc, minAlloc, maxAlloc, planName, replanner, true, context);
+    Plan plan =
+            new InMemoryPlan(queueMetrics, policy, agent, totalCapacity, 1L,
+                    resCalc, minAlloc, maxAlloc, planName, replanner, true, context);
     ReservationId reservationID =
-        ReservationSystemTestUtil.getNewReservationId();
-    int[] alloc = { 10, 10, 10, 10, 10, 10 };
+            ReservationSystemTestUtil.getNewReservationId();
+    int[] alloc = {10, 10, 10, 10, 10, 10};
     int start = 100;
-    ReservationAllocation rAllocation =
-        createReservationAllocation(reservationID, start, alloc);
+    ReservationAllocation rAllocation = createReservationAllocation
+            (reservationID, start, alloc);
     Assert.assertNull(plan.getReservationById(reservationID));
     try {
       plan.addReservation(rAllocation, false);
@@ -674,22 +549,23 @@ public class TestInMemoryPlan {
     }
 
     Set<ReservationAllocation> rAllocations =
-        plan.getReservationsAtTime(rAllocation.getStartTime());
+            plan.getReservationsAtTime(rAllocation.getStartTime());
     Assert.assertTrue(rAllocations.size() == 1);
-    Assert.assertTrue(rAllocation
-        .compareTo((ReservationAllocation) rAllocations.toArray()[0]) == 0);
+    Assert.assertTrue(rAllocation.compareTo(
+            (ReservationAllocation) rAllocations.toArray()[0]) == 0);
   }
 
   @Test
   public void testGetReservationsWithNoInput() {
-    Plan plan = new InMemoryPlan(queueMetrics, policy, agent, totalCapacity, 1L,
-        resCalc, minAlloc, maxAlloc, planName, replanner, true, context);
+    Plan plan =
+            new InMemoryPlan(queueMetrics, policy, agent, totalCapacity, 1L,
+                    resCalc, minAlloc, maxAlloc, planName, replanner, true, context);
     ReservationId reservationID =
-        ReservationSystemTestUtil.getNewReservationId();
-    int[] alloc = { 10, 10, 10, 10, 10, 10 };
+            ReservationSystemTestUtil.getNewReservationId();
+    int[] alloc = {10, 10, 10, 10, 10, 10};
     int start = 100;
-    ReservationAllocation rAllocation =
-        createReservationAllocation(reservationID, start, alloc);
+    ReservationAllocation rAllocation = createReservationAllocation
+            (reservationID, start, alloc);
     Assert.assertNull(plan.getReservationById(reservationID));
     try {
       plan.addReservation(rAllocation, false);
@@ -700,21 +576,22 @@ public class TestInMemoryPlan {
     // Verify that getReservations defaults to getting all reservations if no
     // reservationID, time interval, and user is provided,
     Set<ReservationAllocation> rAllocations =
-        plan.getReservations(null, null, "");
+            plan.getReservations(null, null, "");
     Assert.assertTrue(rAllocations.size() == 1);
-    Assert.assertTrue(rAllocation
-        .compareTo((ReservationAllocation) rAllocations.toArray()[0]) == 0);
+    Assert.assertTrue(rAllocation.compareTo(
+            (ReservationAllocation) rAllocations.toArray()[0]) == 0);
   }
 
   @Test
   public void testGetReservationsWithNoReservation() {
-    Plan plan = new InMemoryPlan(queueMetrics, policy, agent, totalCapacity, 1L,
-        resCalc, minAlloc, maxAlloc, planName, replanner, true, context);
+    Plan plan =
+            new InMemoryPlan(queueMetrics, policy, agent, totalCapacity, 1L,
+                    resCalc, minAlloc, maxAlloc, planName, replanner, true, context);
     // Verify that get reservation returns no entries if no queries are made.
 
     ReservationInterval interval = new ReservationInterval(0, Long.MAX_VALUE);
     Set<ReservationAllocation> rAllocations =
-        plan.getReservations(null, interval, "");
+            plan.getReservations(null, interval, "");
     Assert.assertTrue(rAllocations.size() == 0);
   }
 
@@ -723,9 +600,7 @@ public class TestInMemoryPlan {
     Assert.assertNotNull(plan.getReservationById(reservationID));
     Assert.assertEquals(rAllocation, plan.getReservationById(reservationID));
     Assert.assertTrue(((InMemoryPlan) plan).getAllReservations().size() == 1);
-    if (rAllocation.getPeriodicity() <= 0) {
-      Assert.assertEquals(rAllocation.getEndTime(), plan.getLastEndTime());
-    }
+    Assert.assertEquals(rAllocation.getEndTime(), plan.getLastEndTime());
     Assert.assertEquals(totalCapacity, plan.getTotalCapacity());
     Assert.assertEquals(minAlloc, plan.getMinimumAllocation());
     Assert.assertEquals(maxAlloc, plan.getMaximumAllocation());
@@ -735,8 +610,7 @@ public class TestInMemoryPlan {
   }
 
   private ReservationDefinition createSimpleReservationDefinition(long arrival,
-      long deadline, long duration, Collection<ReservationRequest> resources,
-      String recurrenceExpression) {
+      long deadline, long duration, Collection<ReservationRequest> resources) {
     // create a request with a single atomic ask
     ReservationDefinition rDef = new ReservationDefinitionPBImpl();
     ReservationRequests reqs = new ReservationRequestsPBImpl();
@@ -745,7 +619,6 @@ public class TestInMemoryPlan {
     rDef.setReservationRequests(reqs);
     rDef.setArrival(arrival);
     rDef.setDeadline(deadline);
-    rDef.setRecurrenceExpression(recurrenceExpression);
     return rDef;
   }
 
@@ -760,43 +633,31 @@ public class TestInMemoryPlan {
       } else {
         numContainers = alloc[i];
       }
-      ReservationRequest rr = ReservationRequest
-          .newInstance(Resource.newInstance(1024, 1), (numContainers));
+      ReservationRequest rr =
+          ReservationRequest.newInstance(Resource.newInstance(1024, 1),
+              (numContainers));
       req.put(new ReservationInterval(startTime + i, startTime + i + 1), rr);
     }
     return req;
   }
 
-  private ReservationAllocation createReservationAllocation(
-      ReservationId reservationID, int start, int[] alloc) {
-    return createReservationAllocation(reservationID, start, alloc, false, "0");
-  }
-
-  private ReservationAllocation createReservationAllocation(
-      ReservationId reservationID, int start, int[] alloc, boolean isStep) {
-    return createReservationAllocation(reservationID, start, alloc, isStep,
-        "0");
-  }
-
-  private ReservationAllocation createReservationAllocation(
-      ReservationId reservationID, int start, int[] alloc,
-      String recurrenceExp) {
-    return createReservationAllocation(reservationID, start, alloc, false,
-        recurrenceExp);
+  private ReservationAllocation createReservationAllocation(ReservationId
+            reservationID, int start, int[] alloc) {
+    return createReservationAllocation(reservationID, start, alloc, false);
   }
 
-  private ReservationAllocation createReservationAllocation(
-      ReservationId reservationID, int start, int[] alloc, boolean isStep,
-      String recurrenceExp) {
+  private ReservationAllocation createReservationAllocation(ReservationId
+            reservationID, int start, int[] alloc, boolean isStep) {
     Map<ReservationInterval, ReservationRequest> allocations =
-        generateAllocation(start, alloc, isStep);
+            generateAllocation(start, alloc, isStep);
     ReservationDefinition rDef =
-        createSimpleReservationDefinition(start, start + alloc.length,
-            alloc.length, allocations.values(), recurrenceExp);
+            createSimpleReservationDefinition(start, start + alloc.length,
+                    alloc.length, allocations.values());
     Map<ReservationInterval, Resource> allocs =
-        ReservationSystemUtil.toResources(allocations);
+            ReservationSystemUtil.toResources(allocations);
     return new InMemoryReservationAllocation(reservationID, rDef, user,
-        planName, start, start + alloc.length, allocs, resCalc, minAlloc);
+            planName,
+                    start, start + alloc.length, allocs, resCalc, minAlloc);
   }
 
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e3345e98/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestPeriodicRLESparseResourceAllocation.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/reservation/TestPeriodicRLESparseResourceAllocation.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestPeriodicRLESparseResourceAllocation.java
index 457e2ee..554eb58 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestPeriodicRLESparseResourceAllocation.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestPeriodicRLESparseResourceAllocation.java
@@ -19,27 +19,26 @@
 package org.apache.hadoop.yarn.server.resourcemanager.reservation;
 
 import org.apache.hadoop.yarn.api.records.Resource;
-import org.apache.hadoop.yarn.server.resourcemanager.reservation.exceptions.PlanningException;
 import org.junit.Assert;
 import org.junit.Test;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 /**
- * Testing the class {@link PeriodicRLESparseResourceAllocation}.
+ * Testing the class PeriodicRLESparseResourceAllocation.
  */
-@SuppressWarnings("checkstyle:nowhitespaceafter")
 public class TestPeriodicRLESparseResourceAllocation {
 
-  private static final Logger LOG =
-      LoggerFactory.getLogger(TestPeriodicRLESparseResourceAllocation.class);
+  private static final Logger LOG = LoggerFactory
+      .getLogger(TestPeriodicRLESparseResourceAllocation.class);
 
   @Test
   public void testPeriodicCapacity() {
-    int[] alloc = { 10, 7, 5, 2, 0 };
-    long[] timeSteps = { 0L, 5L, 10L, 15L, 19L };
-    RLESparseResourceAllocation rleSparseVector = ReservationSystemTestUtil
-        .generateRLESparseResourceAllocation(alloc, timeSteps);
+    int[] alloc = {10, 7, 5, 2, 0};
+    long[] timeSteps = {0L, 5L, 10L, 15L, 19L};
+    RLESparseResourceAllocation rleSparseVector =
+        ReservationSystemTestUtil.generateRLESparseResourceAllocation(
+            alloc, timeSteps);
     PeriodicRLESparseResourceAllocation periodicVector =
         new PeriodicRLESparseResourceAllocation(rleSparseVector, 20L);
     LOG.info(periodicVector.toString());
@@ -55,74 +54,43 @@ public class TestPeriodicRLESparseResourceAllocation {
 
   @Test
   public void testMaxPeriodicCapacity() {
-    int[] alloc = { 2, 5, 7, 10, 3, 4, 6, 8 };
-    long[] timeSteps = { 0L, 1L, 2L, 3L, 4L, 5L, 6L, 7L };
-    RLESparseResourceAllocation rleSparseVector = ReservationSystemTestUtil
-        .generateRLESparseResourceAllocation(alloc, timeSteps);
+    int[] alloc = {2, 5, 7, 10, 3, 4, 6, 8};
+    long[] timeSteps = {0L, 1L, 2L, 3L, 4L, 5L, 6L, 7L};
+    RLESparseResourceAllocation rleSparseVector =
+        ReservationSystemTestUtil.generateRLESparseResourceAllocation(
+            alloc, timeSteps);
     PeriodicRLESparseResourceAllocation periodicVector =
         new PeriodicRLESparseResourceAllocation(rleSparseVector, 8L);
     LOG.info(periodicVector.toString());
-    Assert.assertEquals(periodicVector.getMaximumPeriodicCapacity(0, 1),
+    Assert.assertEquals(
+        periodicVector.getMaximumPeriodicCapacity(0, 1),
         Resource.newInstance(10, 10));
-    Assert.assertEquals(periodicVector.getMaximumPeriodicCapacity(8, 2),
+    Assert.assertEquals(
+        periodicVector.getMaximumPeriodicCapacity(8, 2),
         Resource.newInstance(7, 7));
-    Assert.assertEquals(periodicVector.getMaximumPeriodicCapacity(16, 3),
+    Assert.assertEquals(
+        periodicVector.getMaximumPeriodicCapacity(16, 3),
         Resource.newInstance(10, 10));
-    Assert.assertEquals(periodicVector.getMaximumPeriodicCapacity(17, 4),
+    Assert.assertEquals(
+        periodicVector.getMaximumPeriodicCapacity(17, 4),
         Resource.newInstance(5, 5));
-    Assert.assertEquals(periodicVector.getMaximumPeriodicCapacity(32, 5),
+    Assert.assertEquals(
+        periodicVector.getMaximumPeriodicCapacity(32, 5),
         Resource.newInstance(4, 4));
   }
 
   @Test
-  public void testMixPeriodicAndNonPeriodic() throws PlanningException {
-    int[] alloc = { 2, 5, 0 };
-    long[] timeSteps = { 1L, 2L, 3L };
-    RLESparseResourceAllocation tempPeriodic = ReservationSystemTestUtil
-        .generateRLESparseResourceAllocation(alloc, timeSteps);
-    PeriodicRLESparseResourceAllocation periodic =
-        new PeriodicRLESparseResourceAllocation(tempPeriodic, 10L);
-
-    int[] alloc2 = { 10, 10, 0 };
-    long[] timeSteps2 = { 12L, 13L, 14L };
-    RLESparseResourceAllocation nonPeriodic = ReservationSystemTestUtil
-        .generateRLESparseResourceAllocation(alloc2, timeSteps2);
-
-    RLESparseResourceAllocation merged =
-        RLESparseResourceAllocation.merge(nonPeriodic.getResourceCalculator(),
-            Resource.newInstance(100 * 1024, 100), periodic, nonPeriodic,
-            RLESparseResourceAllocation.RLEOperator.add, 2, 25);
-
-    Assert.assertEquals(Resource.newInstance(5, 5),
-        merged.getCapacityAtTime(2L));
-    Assert.assertEquals(Resource.newInstance(0, 0),
-        merged.getCapacityAtTime(3L));
-    Assert.assertEquals(Resource.newInstance(2, 2),
-        merged.getCapacityAtTime(11L));
-    Assert.assertEquals(Resource.newInstance(15, 15),
-        merged.getCapacityAtTime(12L));
-    Assert.assertEquals(Resource.newInstance(10, 10),
-        merged.getCapacityAtTime(13L));
-    Assert.assertEquals(Resource.newInstance(0, 0),
-        merged.getCapacityAtTime(14L));
-    Assert.assertEquals(Resource.newInstance(2, 2),
-        merged.getCapacityAtTime(21L));
-    Assert.assertEquals(Resource.newInstance(5, 5),
-        merged.getCapacityAtTime(22L));
-    Assert.assertEquals(Resource.newInstance(0, 0),
-        merged.getCapacityAtTime(23L));
-  }
-
-  @Test
   public void testSetCapacityInInterval() {
-    int[] alloc = { 2, 5, 0 };
-    long[] timeSteps = { 1L, 2L, 3L };
-    RLESparseResourceAllocation rleSparseVector = ReservationSystemTestUtil
-        .generateRLESparseResourceAllocation(alloc, timeSteps);
+    int[] alloc = {2, 5, 0};
+    long[] timeSteps = {1L, 2L, 3L};
+    RLESparseResourceAllocation rleSparseVector =
+        ReservationSystemTestUtil.generateRLESparseResourceAllocation(
+            alloc, timeSteps);
     PeriodicRLESparseResourceAllocation periodicVector =
         new PeriodicRLESparseResourceAllocation(rleSparseVector, 10L);
     ReservationInterval interval = new ReservationInterval(5L, 10L);
-    periodicVector.addInterval(interval, Resource.newInstance(8, 8));
+    periodicVector.addInterval(
+        interval, Resource.newInstance(8, 8));
     Assert.assertEquals(Resource.newInstance(8, 8),
         periodicVector.getCapacityAtTime(5L));
     Assert.assertEquals(Resource.newInstance(8, 8),
@@ -131,20 +99,21 @@ public class TestPeriodicRLESparseResourceAllocation {
         periodicVector.getCapacityAtTime(10L));
     Assert.assertEquals(Resource.newInstance(0, 0),
         periodicVector.getCapacityAtTime(0L));
-    // Assert.assertFalse(periodicVector.addInterval(
-    // new ReservationInterval(7L, 12L), Resource.newInstance(8, 8)));
+    Assert.assertFalse(periodicVector.addInterval(
+        new ReservationInterval(7L, 12L), Resource.newInstance(8, 8)));
   }
 
   public void testRemoveInterval() {
-    int[] alloc = { 2, 5, 3, 4, 0 };
-    long[] timeSteps = { 1L, 3L, 5L, 7L, 9L };
-    RLESparseResourceAllocation rleSparseVector = ReservationSystemTestUtil
-        .generateRLESparseResourceAllocation(alloc, timeSteps);
+    int[] alloc = {2, 5, 3, 4, 0};
+    long[] timeSteps = {1L, 3L, 5L, 7L, 9L};
+    RLESparseResourceAllocation rleSparseVector =
+        ReservationSystemTestUtil.generateRLESparseResourceAllocation(
+            alloc, timeSteps);
     PeriodicRLESparseResourceAllocation periodicVector =
         new PeriodicRLESparseResourceAllocation(rleSparseVector, 10L);
     ReservationInterval interval = new ReservationInterval(3L, 7L);
-    Assert.assertTrue(
-        periodicVector.removeInterval(interval, Resource.newInstance(3, 3)));
+    Assert.assertTrue(periodicVector.removeInterval(
+        interval, Resource.newInstance(3, 3)));
     Assert.assertEquals(Resource.newInstance(2, 2),
         periodicVector.getCapacityAtTime(1L));
     Assert.assertEquals(Resource.newInstance(2, 2),

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e3345e98/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestRLESparseResourceAllocation.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/reservation/TestRLESparseResourceAllocation.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestRLESparseResourceAllocation.java
index 0027ceb..bfe46e1 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestRLESparseResourceAllocation.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestRLESparseResourceAllocation.java
@@ -40,14 +40,10 @@ import org.junit.Test;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-/**
- * Testing the class {@link RLESparseResourceAllocation}.
- */
-@SuppressWarnings("checkstyle:nowhitespaceafter")
 public class TestRLESparseResourceAllocation {
 
-  private static final Logger LOG =
-      LoggerFactory.getLogger(TestRLESparseResourceAllocation.class);
+  private static final Logger LOG = LoggerFactory
+      .getLogger(TestRLESparseResourceAllocation.class);
 
   @Test
   public void testMergeAdd() throws PlanningException {
@@ -200,8 +196,7 @@ public class TestRLESparseResourceAllocation {
       // Expected!
     }
 
-    // Testing that the subtractTestNonNegative detects problems even if only
-    // one
+    // Testing that the subtractTestNonNegative detects problems even if only one
     // of the resource dimensions is "<0"
     a.put(10L, Resource.newInstance(10, 5));
     b.put(11L, Resource.newInstance(5, 6));
@@ -291,8 +286,9 @@ public class TestRLESparseResourceAllocation {
   public void testRangeOverlapping() {
     ResourceCalculator resCalc = new DefaultResourceCalculator();
 
-    RLESparseResourceAllocation r = new RLESparseResourceAllocation(resCalc);
-    int[] alloc = { 10, 10, 10, 10, 10, 10 };
+    RLESparseResourceAllocation r =
+        new RLESparseResourceAllocation(resCalc);
+    int[] alloc = {10, 10, 10, 10, 10, 10};
     int start = 100;
     Set<Entry<ReservationInterval, Resource>> inputs =
         generateAllocation(start, alloc, false).entrySet();
@@ -303,9 +299,9 @@ public class TestRLESparseResourceAllocation {
     long d = r.getLatestNonNullTime();
 
     // tries to trigger "out-of-range" bug
-    r = r.getRangeOverlapping(s, d);
-    r = r.getRangeOverlapping(s - 1, d - 1);
-    r = r.getRangeOverlapping(s + 1, d + 1);
+    r =  r.getRangeOverlapping(s, d);
+    r = r.getRangeOverlapping(s-1, d-1);
+    r = r.getRangeOverlapping(s+1, d+1);
   }
 
   @Test
@@ -374,29 +370,25 @@ public class TestRLESparseResourceAllocation {
     // Current bug prevents this to pass. The RLESparseResourceAllocation
     // does not handle removal of "partial"
     // allocations correctly.
-    Assert.assertEquals(102400,
-        rleSparseVector.getCapacityAtTime(10).getMemorySize());
-    Assert.assertEquals(0,
-        rleSparseVector.getCapacityAtTime(13).getMemorySize());
-    Assert.assertEquals(0,
-        rleSparseVector.getCapacityAtTime(19).getMemorySize());
-    Assert.assertEquals(102400,
-        rleSparseVector.getCapacityAtTime(21).getMemorySize());
-    Assert.assertEquals(2 * 102400,
-        rleSparseVector.getCapacityAtTime(26).getMemorySize());
+    Assert.assertEquals(102400, rleSparseVector.getCapacityAtTime(10)
+        .getMemorySize());
+    Assert.assertEquals(0, rleSparseVector.getCapacityAtTime(13).getMemorySize());
+    Assert.assertEquals(0, rleSparseVector.getCapacityAtTime(19).getMemorySize());
+    Assert.assertEquals(102400, rleSparseVector.getCapacityAtTime(21)
+        .getMemorySize());
+    Assert.assertEquals(2 * 102400, rleSparseVector.getCapacityAtTime(26)
+        .getMemorySize());
 
     ReservationInterval riRemove2 = new ReservationInterval(9, 13);
     rleSparseVector.removeInterval(riRemove2, rr);
     LOG.info(rleSparseVector.toString());
 
-    Assert.assertEquals(0,
-        rleSparseVector.getCapacityAtTime(11).getMemorySize());
-    Assert.assertEquals(-102400,
-        rleSparseVector.getCapacityAtTime(9).getMemorySize());
-    Assert.assertEquals(0,
-        rleSparseVector.getCapacityAtTime(13).getMemorySize());
-    Assert.assertEquals(102400,
-        rleSparseVector.getCapacityAtTime(20).getMemorySize());
+    Assert.assertEquals(0, rleSparseVector.getCapacityAtTime(11).getMemorySize());
+    Assert.assertEquals(-102400, rleSparseVector.getCapacityAtTime(9)
+        .getMemorySize());
+    Assert.assertEquals(0, rleSparseVector.getCapacityAtTime(13).getMemorySize());
+    Assert.assertEquals(102400, rleSparseVector.getCapacityAtTime(20)
+        .getMemorySize());
 
   }
 
@@ -508,8 +500,7 @@ public class TestRLESparseResourceAllocation {
     }
     mapAllocations = rleSparseVector.toIntervalMap();
     Assert.assertTrue(mapAllocations.size() == 5);
-    for (Entry<ReservationInterval, Resource> entry : mapAllocations
-        .entrySet()) {
+    for (Entry<ReservationInterval, Resource> entry : mapAllocations.entrySet()) {
       ReservationInterval interval = entry.getKey();
       Resource resource = entry.getValue();
       if (interval.getStartTime() == 101L) {
@@ -535,46 +526,59 @@ public class TestRLESparseResourceAllocation {
 
   @Test
   public void testMaxPeriodicCapacity() {
-    long[] timeSteps = { 0L, 1L, 2L, 3L, 4L, 5L, 6L, 7L };
-    int[] alloc = { 2, 5, 7, 10, 3, 4, 6, 8 };
-    RLESparseResourceAllocation rleSparseVector = ReservationSystemTestUtil
-        .generateRLESparseResourceAllocation(alloc, timeSteps);
+    long[] timeSteps = {0L, 1L, 2L, 3L, 4L, 5L, 6L, 7L};
+    int[] alloc = {2, 5, 7, 10, 3, 4, 6, 8};
+    RLESparseResourceAllocation rleSparseVector =
+        ReservationSystemTestUtil.generateRLESparseResourceAllocation(
+            alloc, timeSteps);
     LOG.info(rleSparseVector.toString());
-    Assert.assertEquals(rleSparseVector.getMaximumPeriodicCapacity(0, 1),
+    Assert.assertEquals(
+        rleSparseVector.getMaximumPeriodicCapacity(0, 1),
         Resource.newInstance(10, 10));
-    Assert.assertEquals(rleSparseVector.getMaximumPeriodicCapacity(0, 2),
+    Assert.assertEquals(
+        rleSparseVector.getMaximumPeriodicCapacity(0, 2),
         Resource.newInstance(7, 7));
-    Assert.assertEquals(rleSparseVector.getMaximumPeriodicCapacity(0, 3),
+    Assert.assertEquals(
+        rleSparseVector.getMaximumPeriodicCapacity(0, 3),
         Resource.newInstance(10, 10));
-    Assert.assertEquals(rleSparseVector.getMaximumPeriodicCapacity(0, 4),
+    Assert.assertEquals(
+        rleSparseVector.getMaximumPeriodicCapacity(0, 4),
         Resource.newInstance(3, 3));
-    Assert.assertEquals(rleSparseVector.getMaximumPeriodicCapacity(0, 5),
+    Assert.assertEquals(
+        rleSparseVector.getMaximumPeriodicCapacity(0, 5),
         Resource.newInstance(4, 4));
-    Assert.assertEquals(rleSparseVector.getMaximumPeriodicCapacity(0, 5),
+    Assert.assertEquals(
+        rleSparseVector.getMaximumPeriodicCapacity(0, 5),
         Resource.newInstance(4, 4));
-    Assert.assertEquals(rleSparseVector.getMaximumPeriodicCapacity(7, 5),
+    Assert.assertEquals(
+        rleSparseVector.getMaximumPeriodicCapacity(7, 5),
         Resource.newInstance(8, 8));
-    Assert.assertEquals(rleSparseVector.getMaximumPeriodicCapacity(10, 3),
+    Assert.assertEquals(
+        rleSparseVector.getMaximumPeriodicCapacity(10, 3),
         Resource.newInstance(0, 0));
   }
 
   @Test
   public void testGetMinimumCapacityInInterval() {
-    long[] timeSteps = { 0L, 1L, 2L, 3L, 4L, 5L, 6L, 7L };
-    int[] alloc = { 2, 5, 7, 10, 3, 4, 0, 8 };
-    RLESparseResourceAllocation rleSparseVector = ReservationSystemTestUtil
-        .generateRLESparseResourceAllocation(alloc, timeSteps);
+    long[] timeSteps = {0L, 1L, 2L, 3L, 4L, 5L, 6L, 7L};
+    int[] alloc = {2, 5, 7, 10, 3, 4, 0, 8};
+    RLESparseResourceAllocation rleSparseVector =
+        ReservationSystemTestUtil.generateRLESparseResourceAllocation(
+            alloc, timeSteps);
     LOG.info(rleSparseVector.toString());
-    Assert.assertEquals(rleSparseVector.getMinimumCapacityInInterval(
-        new ReservationInterval(1L, 3L)), Resource.newInstance(5, 5));
-    Assert.assertEquals(rleSparseVector.getMinimumCapacityInInterval(
-        new ReservationInterval(2L, 5L)), Resource.newInstance(3, 3));
-    Assert.assertEquals(rleSparseVector.getMinimumCapacityInInterval(
-        new ReservationInterval(1L, 7L)), Resource.newInstance(0, 0));
+    Assert.assertEquals(
+        rleSparseVector.getMinimumCapacityInInterval(
+            new ReservationInterval(1L, 3L)), Resource.newInstance(5, 5));
+    Assert.assertEquals(
+        rleSparseVector.getMinimumCapacityInInterval(
+            new ReservationInterval(2L, 5L)), Resource.newInstance(3, 3));
+    Assert.assertEquals(
+        rleSparseVector.getMinimumCapacityInInterval(
+            new ReservationInterval(1L, 7L)), Resource.newInstance(0, 0));
   }
 
-  private void setupArrays(TreeMap<Long, Resource> a,
-      TreeMap<Long, Resource> b) {
+  private void setupArrays(
+      TreeMap<Long, Resource> a, TreeMap<Long, Resource> b) {
     a.put(10L, Resource.newInstance(5, 5));
     a.put(20L, Resource.newInstance(10, 10));
     a.put(30L, Resource.newInstance(15, 15));
@@ -616,8 +620,8 @@ public class TestRLESparseResourceAllocation {
         numContainers = alloc[i];
       }
       req.put(new ReservationInterval(startTime + i, startTime + i + 1),
-          ReservationSystemUtil.toResource(ReservationRequest
-              .newInstance(Resource.newInstance(1024, 1), (numContainers))));
+          ReservationSystemUtil.toResource(ReservationRequest.newInstance(
+              Resource.newInstance(1024, 1), (numContainers))));
     }
     return req;
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e3345e98/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/planning/TestSimpleCapacityReplanner.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/reservation/planning/TestSimpleCapacityReplanner.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/planning/TestSimpleCapacityReplanner.java
index ddd290d..c4f94c2 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/planning/TestSimpleCapacityReplanner.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/planning/TestSimpleCapacityReplanner.java
@@ -31,7 +31,6 @@ import org.apache.hadoop.yarn.api.records.ReservationDefinition;
 import org.apache.hadoop.yarn.api.records.ReservationId;
 import org.apache.hadoop.yarn.api.records.ReservationRequest;
 import org.apache.hadoop.yarn.api.records.Resource;
-import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
 import org.apache.hadoop.yarn.server.resourcemanager.reservation.InMemoryPlan;
 import org.apache.hadoop.yarn.server.resourcemanager.reservation.InMemoryReservationAllocation;
@@ -79,10 +78,9 @@ public class TestSimpleCapacityReplanner {
     enf.init("blah", conf);
 
     // Initialize the plan with more resources
-    InMemoryPlan plan = new InMemoryPlan(queueMetrics, policy, agent,
-        clusterCapacity, step, res, minAlloc, maxAlloc, "dedicated", enf, true,
-        YarnConfiguration.DEFAULT_RM_RESERVATION_SYSTEM_MAX_PERIODICITY,
-        context, clock);
+    InMemoryPlan plan =
+        new InMemoryPlan(queueMetrics, policy, agent, clusterCapacity, step,
+            res, minAlloc, maxAlloc, "dedicated", enf, true, context, clock);
 
     // add reservation filling the plan (separating them 1ms, so we are sure
     // s2 follows s1 on acceptance


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[14/37] hadoop git commit: HDFS-12392. Writing striped file failed due to different cell size. Contributed by Sammi Chen

Posted by ae...@apache.org.
HDFS-12392. Writing striped file failed due to different cell size. Contributed by Sammi Chen


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

Branch: refs/heads/HDFS-7240
Commit: d7f27043ce034d69003b054dfe4f9e8e317ec543
Parents: d4035d4
Author: Kai Zheng <ka...@intel.com>
Authored: Wed Sep 6 13:29:52 2017 +0800
Committer: Kai Zheng <ka...@intel.com>
Committed: Wed Sep 6 13:29:52 2017 +0800

----------------------------------------------------------------------
 .../hadoop/hdfs/DFSStripedOutputStream.java     |  2 +
 .../hadoop/hdfs/TestErasureCodingPolicies.java  | 50 +++++++++++++++++++-
 2 files changed, 50 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/d7f27043/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSStripedOutputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSStripedOutputStream.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSStripedOutputStream.java
index 408b325..7f05338 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSStripedOutputStream.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSStripedOutputStream.java
@@ -215,6 +215,7 @@ public class DFSStripedOutputStream extends DFSOutputStream
       buffers = new ByteBuffer[numAllBlocks];
       for (int i = 0; i < buffers.length; i++) {
         buffers[i] = BUFFER_POOL.getBuffer(useDirectBuffer(), cellSize);
+        buffers[i].limit(cellSize);
       }
     }
 
@@ -237,6 +238,7 @@ public class DFSStripedOutputStream extends DFSOutputStream
     private void clear() {
       for (int i = 0; i< numAllBlocks; i++) {
         buffers[i].clear();
+        buffers[i].limit(cellSize);
       }
     }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d7f27043/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodingPolicies.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodingPolicies.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodingPolicies.java
index 0c545be..e095602 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodingPolicies.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodingPolicies.java
@@ -35,6 +35,7 @@ import org.apache.hadoop.hdfs.client.HdfsAdmin;
 import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
 import org.apache.hadoop.hdfs.server.namenode.INodeFile;
 import org.apache.hadoop.io.erasurecode.ECSchema;
+import org.apache.hadoop.io.erasurecode.ErasureCodeConstants;
 import org.apache.hadoop.security.AccessControlException;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.test.GenericTestUtils;
@@ -60,7 +61,7 @@ public class TestErasureCodingPolicies {
   private Configuration conf;
   private MiniDFSCluster cluster;
   private DistributedFileSystem fs;
-  private static final int BLOCK_SIZE = 1024;
+  private static final int BLOCK_SIZE = 16 * 1024;
   private ErasureCodingPolicy ecPolicy;
   private FSNamesystem namesystem;
 
@@ -78,7 +79,8 @@ public class TestErasureCodingPolicies {
     conf.setInt(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, BLOCK_SIZE);
     DFSTestUtil.enableAllECPolicies(conf);
     cluster = new MiniDFSCluster.Builder(conf).
-        numDataNodes(1).build();
+        numDataNodes(ecPolicy.getNumDataUnits() + ecPolicy.getNumParityUnits()).
+        build();
     cluster.waitActive();
     fs = cluster.getFileSystem();
     namesystem = cluster.getNamesystem();
@@ -836,4 +838,48 @@ public class TestErasureCodingPolicies {
         ecPolicy, fs.getErasureCodingPolicy(subReplicaFile));
     fs.delete(subReplicaFile, false);
   }
+
+  @Test
+  public void testDifferentErasureCodingPolicyCellSize() throws Exception {
+    // add policy with cell size 8K
+    ErasureCodingPolicy newPolicy1 =
+        new ErasureCodingPolicy(ErasureCodeConstants.RS_3_2_SCHEMA, 8 * 1024);
+    ErasureCodingPolicy[] policyArray =
+        new ErasureCodingPolicy[] {newPolicy1};
+    AddECPolicyResponse[] responses = fs.addErasureCodingPolicies(policyArray);
+    assertEquals(1, responses.length);
+    assertTrue(responses[0].isSucceed());
+    newPolicy1 = responses[0].getPolicy();
+
+    // add policy with cell size 4K
+    ErasureCodingPolicy newPolicy2 =
+        new ErasureCodingPolicy(ErasureCodeConstants.RS_3_2_SCHEMA, 4 * 1024);
+    policyArray = new ErasureCodingPolicy[] {newPolicy2};
+    responses = fs.addErasureCodingPolicies(policyArray);
+    assertEquals(1, responses.length);
+    assertTrue(responses[0].isSucceed());
+    newPolicy2 = responses[0].getPolicy();
+
+    // enable policies
+    fs.enableErasureCodingPolicy(newPolicy1.getName());
+    fs.enableErasureCodingPolicy(newPolicy2.getName());
+
+    final Path stripedDir1 = new Path("/striped1");
+    final Path stripedDir2 = new Path("/striped2");
+    final Path file1 = new Path(stripedDir1, "file");
+    final Path file2 = new Path(stripedDir2, "file");
+
+    fs.mkdirs(stripedDir1);
+    fs.setErasureCodingPolicy(stripedDir1, newPolicy1.getName());
+    fs.mkdirs(stripedDir2);
+    fs.setErasureCodingPolicy(stripedDir2, newPolicy2.getName());
+
+    final int fileLength = BLOCK_SIZE * newPolicy1.getNumDataUnits();
+    final byte[] bytes = StripedFileTestUtil.generateBytes(fileLength);
+    DFSTestUtil.writeFile(fs, file1, bytes);
+    DFSTestUtil.writeFile(fs, file2, bytes);
+
+    fs.delete(stripedDir1, true);
+    fs.delete(stripedDir2, true);
+  }
 }


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[02/37] hadoop git commit: Plan/ResourceAllocation data structure enhancements required to support recurring reservations in ReservationSystem.

Posted by ae...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/7996eca7/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/RLESparseResourceAllocation.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/reservation/RLESparseResourceAllocation.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/RLESparseResourceAllocation.java
index 658387b..100d38c 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/RLESparseResourceAllocation.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/RLESparseResourceAllocation.java
@@ -18,8 +18,7 @@
 
 package org.apache.hadoop.yarn.server.resourcemanager.reservation;
 
-import java.io.IOException;
-import java.io.StringWriter;
+import java.util.Collections;
 import java.util.Iterator;
 import java.util.Map;
 import java.util.Map.Entry;
@@ -33,8 +32,6 @@ import org.apache.hadoop.yarn.server.resourcemanager.reservation.exceptions.Plan
 import org.apache.hadoop.yarn.util.resource.ResourceCalculator;
 import org.apache.hadoop.yarn.util.resource.Resources;
 
-import com.google.gson.stream.JsonWriter;
-
 /**
  * This is a run length encoded sparse data structure that maintains resource
  * allocations over time.
@@ -44,12 +41,14 @@ public class RLESparseResourceAllocation {
   private static final int THRESHOLD = 100;
   private static final Resource ZERO_RESOURCE = Resources.none();
 
-  private NavigableMap<Long, Resource> cumulativeCapacity =
+  @SuppressWarnings("checkstyle:visibilitymodifier")
+  protected NavigableMap<Long, Resource> cumulativeCapacity =
       new TreeMap<Long, Resource>();
 
   private final ReentrantReadWriteLock readWriteLock =
       new ReentrantReadWriteLock();
-  private final Lock readLock = readWriteLock.readLock();
+  @SuppressWarnings("checkstyle:visibilitymodifier")
+  protected final Lock readLock = readWriteLock.readLock();
   private final Lock writeLock = readWriteLock.writeLock();
 
   private final ResourceCalculator resourceCalculator;
@@ -236,34 +235,6 @@ public class RLESparseResourceAllocation {
   }
 
   /**
-   * Returns the JSON string representation of the current resources allocated
-   * over time.
-   *
-   * @return the JSON string representation of the current resources allocated
-   *         over time
-   */
-  public String toMemJSONString() {
-    StringWriter json = new StringWriter();
-    JsonWriter jsonWriter = new JsonWriter(json);
-    readLock.lock();
-    try {
-      jsonWriter.beginObject();
-      // jsonWriter.name("timestamp").value("resource");
-      for (Map.Entry<Long, Resource> r : cumulativeCapacity.entrySet()) {
-        jsonWriter.name(r.getKey().toString()).value(r.getValue().toString());
-      }
-      jsonWriter.endObject();
-      jsonWriter.close();
-      return json.toString();
-    } catch (IOException e) {
-      // This should not happen
-      return "";
-    } finally {
-      readLock.unlock();
-    }
-  }
-
-  /**
    * Returns the representation of the current resources allocated over time as
    * an interval map (in the defined non-null range).
    *
@@ -304,7 +275,7 @@ public class RLESparseResourceAllocation {
   public NavigableMap<Long, Resource> getCumulative() {
     readLock.lock();
     try {
-      return cumulativeCapacity;
+      return Collections.unmodifiableNavigableMap(cumulativeCapacity);
     } finally {
       readLock.unlock();
     }
@@ -437,8 +408,8 @@ public class RLESparseResourceAllocation {
       Resource val = Resources.negate(e.getValue());
       // test for negative value and throws
       if (operator == RLEOperator.subtractTestNonNegative
-          && (Resources.fitsIn(val, ZERO_RESOURCE) &&
-              !Resources.equals(val, ZERO_RESOURCE))) {
+          && (Resources.fitsIn(val, ZERO_RESOURCE)
+              && !Resources.equals(val, ZERO_RESOURCE))) {
         throw new PlanningException(
             "RLESparseResourceAllocation: merge failed as the "
                 + "resulting RLESparseResourceAllocation would be negative");
@@ -504,22 +475,29 @@ public class RLESparseResourceAllocation {
 
   }
 
+  /**
+   * Get a {@link RLESparseResourceAllocation} view of the {@link Resource}
+   * allocations between the specified start and end times.
+   *
+   * @param start the time from which the {@link Resource} allocations are
+   *          required
+   * @param end the time upto which the {@link Resource} allocations are
+   *          required
+   * @return the overlapping allocations
+   */
   public RLESparseResourceAllocation getRangeOverlapping(long start, long end) {
     readLock.lock();
     try {
       NavigableMap<Long, Resource> a = this.getCumulative();
-
       if (a != null && !a.isEmpty()) {
         // include the portion of previous entry that overlaps start
         if (start > a.firstKey()) {
           long previous = a.floorKey(start);
           a = a.tailMap(previous, true);
         }
-
         if (end < a.lastKey()) {
           a = a.headMap(end, true);
         }
-
       }
       RLESparseResourceAllocation ret =
           new RLESparseResourceAllocation(a, resourceCalculator);
@@ -527,7 +505,33 @@ public class RLESparseResourceAllocation {
     } finally {
       readLock.unlock();
     }
+  }
 
+  /**
+   * This method shifts all the timestamp of the {@link Resource} entries by the
+   * specified "delta".
+   *
+   * @param delta the time by which to shift the {@link Resource} allocations
+   */
+  public void shift(long delta) {
+    writeLock.lock();
+    try {
+      TreeMap<Long, Resource> newCum = new TreeMap<>();
+      long start;
+      for (Map.Entry<Long, Resource> entry : cumulativeCapacity.entrySet()) {
+        if (delta > 0) {
+          start = (entry.getKey() == Long.MAX_VALUE) ? Long.MAX_VALUE
+              : entry.getKey() + delta;
+        } else {
+          start = (entry.getKey() == Long.MIN_VALUE) ? Long.MIN_VALUE
+              : entry.getKey() + delta;
+        }
+        newCum.put(start, entry.getValue());
+      }
+      cumulativeCapacity = newCum;
+    } finally {
+      writeLock.unlock();
+    }
   }
 
   /**
@@ -541,8 +545,8 @@ public class RLESparseResourceAllocation {
   /**
    * Get the maximum capacity across specified time instances. The search-space
    * is specified using the starting value, tick, and the periodic interval for
-   * search. Maximum resource allocation across tick, tick + period,
-   * tick + 2 * period,..., tick + n * period .. is returned.
+   * search. Maximum resource allocation across tick, tick + period, tick + 2 *
+   * period,..., tick + n * period .. is returned.
    *
    * @param tick the starting time instance
    * @param period interval at which capacity is evaluated
@@ -550,14 +554,19 @@ public class RLESparseResourceAllocation {
    */
   public Resource getMaximumPeriodicCapacity(long tick, long period) {
     Resource maxCapacity = ZERO_RESOURCE;
-    if (!cumulativeCapacity.isEmpty()) {
-      Long lastKey = cumulativeCapacity.lastKey();
-      for (long t = tick; t <= lastKey; t = t + period) {
-        maxCapacity = Resources.componentwiseMax(maxCapacity,
-            cumulativeCapacity.floorEntry(t).getValue());
+    readLock.lock();
+    try {
+      if (!cumulativeCapacity.isEmpty()) {
+        Long lastKey = cumulativeCapacity.lastKey();
+        for (long t = tick; t <= lastKey; t = t + period) {
+          maxCapacity = Resources.componentwiseMax(maxCapacity,
+              cumulativeCapacity.floorEntry(t).getValue());
+        }
       }
+      return maxCapacity;
+    } finally {
+      readLock.unlock();
     }
-    return maxCapacity;
   }
 
   /**
@@ -567,17 +576,17 @@ public class RLESparseResourceAllocation {
    * @return minimum resource allocation
    */
   public Resource getMinimumCapacityInInterval(ReservationInterval interval) {
-    Resource minCapacity = Resource.newInstance(
-        Integer.MAX_VALUE, Integer.MAX_VALUE);
+    Resource minCapacity =
+        Resource.newInstance(Integer.MAX_VALUE, Integer.MAX_VALUE);
     long start = interval.getStartTime();
     long end = interval.getEndTime();
     NavigableMap<Long, Resource> capacityRange =
-        this.getRangeOverlapping(start, end).getCumulative();
+        getRangeOverlapping(start, end).getCumulative();
     if (!capacityRange.isEmpty()) {
       for (Map.Entry<Long, Resource> entry : capacityRange.entrySet()) {
         if (entry.getValue() != null) {
-          minCapacity = Resources.componentwiseMin(minCapacity,
-              entry.getValue());
+          minCapacity =
+              Resources.componentwiseMin(minCapacity, entry.getValue());
         }
       }
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7996eca7/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/ReservationAllocation.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/reservation/ReservationAllocation.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/ReservationAllocation.java
index 0da95ac..bb4a7fb 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/ReservationAllocation.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/ReservationAllocation.java
@@ -24,14 +24,16 @@ import org.apache.hadoop.yarn.api.records.ReservationDefinition;
 import org.apache.hadoop.yarn.api.records.ReservationId;
 import org.apache.hadoop.yarn.api.records.Resource;
 
+import com.google.common.annotations.VisibleForTesting;
+
 /**
  * A ReservationAllocation represents a concrete allocation of resources over
  * time that satisfy a certain {@link ReservationDefinition}. This is used
  * internally by a {@link Plan} to store information about how each of the
  * accepted {@link ReservationDefinition} have been allocated.
  */
-public interface ReservationAllocation extends
-    Comparable<ReservationAllocation> {
+public interface ReservationAllocation
+    extends Comparable<ReservationAllocation> {
 
   /**
    * Returns the unique identifier {@link ReservationId} that represents the
@@ -40,28 +42,28 @@ public interface ReservationAllocation extends
    * @return reservationId the unique identifier {@link ReservationId} that
    *         represents the reservation
    */
-  public ReservationId getReservationId();
+  ReservationId getReservationId();
 
   /**
    * Returns the original {@link ReservationDefinition} submitted by the client
    * 
    * @return the {@link ReservationDefinition} submitted by the client
    */
-  public ReservationDefinition getReservationDefinition();
+  ReservationDefinition getReservationDefinition();
 
   /**
    * Returns the time at which the reservation is activated.
    * 
    * @return the time at which the reservation is activated
    */
-  public long getStartTime();
+  long getStartTime();
 
   /**
    * Returns the time at which the reservation terminates.
    * 
    * @return the time at which the reservation terminates
    */
-  public long getEndTime();
+  long getEndTime();
 
   /**
    * Returns the map of resources requested against the time interval for which
@@ -70,28 +72,28 @@ public interface ReservationAllocation extends
    * @return the allocationRequests the map of resources requested against the
    *         time interval for which they were
    */
-  public Map<ReservationInterval, Resource> getAllocationRequests();
+  Map<ReservationInterval, Resource> getAllocationRequests();
 
   /**
    * Return a string identifying the plan to which the reservation belongs
    * 
    * @return the plan to which the reservation belongs
    */
-  public String getPlanName();
+  String getPlanName();
 
   /**
    * Returns the user who requested the reservation
    * 
    * @return the user who requested the reservation
    */
-  public String getUser();
+  String getUser();
 
   /**
    * Returns whether the reservation has gang semantics or not
    * 
    * @return true if there is a gang request, false otherwise
    */
-  public boolean containsGangs();
+  boolean containsGangs();
 
   /**
    * Sets the time at which the reservation was accepted by the system
@@ -99,14 +101,14 @@ public interface ReservationAllocation extends
    * @param acceptedAt the time at which the reservation was accepted by the
    *          system
    */
-  public void setAcceptanceTimestamp(long acceptedAt);
+  void setAcceptanceTimestamp(long acceptedAt);
 
   /**
    * Returns the time at which the reservation was accepted by the system
    * 
    * @return the time at which the reservation was accepted by the system
    */
-  public long getAcceptanceTime();
+  long getAcceptanceTime();
 
   /**
    * Returns the capacity represented by cumulative resources reserved by the
@@ -116,12 +118,42 @@ public interface ReservationAllocation extends
    *          requested
    * @return the resources reserved at the specified time
    */
-  public Resource getResourcesAtTime(long tick);
+  Resource getResourcesAtTime(long tick);
+
+  /**
+   * Return a RLE representation of used resources.
+   *
+   * @return a RLE encoding of resources allocated over time.
+   */
+  RLESparseResourceAllocation getResourcesOverTime();
+
 
   /**
    * Return a RLE representation of used resources.
+   *
+   * @param start start of the time interval.
+   * @param end end of the time interval.
    * @return a RLE encoding of resources allocated over time.
    */
-  public RLESparseResourceAllocation getResourcesOverTime();
+  RLESparseResourceAllocation getResourcesOverTime(long start, long end);
+
+  /**
+   * Get the periodicity of this reservation representing the time period of the
+   * periodic job. Period is represented in milliseconds for periodic jobs.
+   * Period is 0 for non-periodic jobs.
+   *
+   * @return periodicity of this reservation
+   */
+  long getPeriodicity();
+
+  /**
+   * Set the periodicity of this reservation representing the time period of the
+   * periodic job. Period is represented in milliseconds for periodic jobs.
+   * Period is 0 for non-periodic jobs.
+   *
+   * @param period periodicity of this reservation
+   */
+  @VisibleForTesting
+  void setPeriodicity(long period);
 
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7996eca7/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/ReservationInputValidator.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/reservation/ReservationInputValidator.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/ReservationInputValidator.java
index 027d066..a66d222 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/ReservationInputValidator.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/ReservationInputValidator.java
@@ -44,6 +44,8 @@ public class ReservationInputValidator {
 
   /**
    * Utility class to validate reservation requests.
+   *
+   * @param clock the {@link Clock} to use
    */
   public ReservationInputValidator(Clock clock) {
     this.clock = clock;
@@ -53,22 +55,21 @@ public class ReservationInputValidator {
       ReservationId reservationId, String auditConstant) throws YarnException {
     // check if the reservation id is valid
     if (reservationId == null) {
-      String message =
-          "Missing reservation id."
-              + " Please try again by specifying a reservation id.";
+      String message = "Missing reservation id."
+          + " Please try again by specifying a reservation id.";
       RMAuditLogger.logFailure("UNKNOWN", auditConstant,
           "validate reservation input", "ClientRMService", message);
       throw RPCUtil.getRemoteException(message);
     }
     String queue = reservationSystem.getQueueForReservation(reservationId);
     String nullQueueErrorMessage =
-            "The specified reservation with ID: " + reservationId
-                    + " is unknown. Please try again with a valid reservation.";
+        "The specified reservation with ID: " + reservationId
+            + " is unknown. Please try again with a valid reservation.";
     String nullPlanErrorMessage = "The specified reservation: " + reservationId
-                            + " is not associated with any valid plan."
-                            + " Please try again with a valid reservation.";
+        + " is not associated with any valid plan."
+        + " Please try again with a valid reservation.";
     return getPlanFromQueue(reservationSystem, queue, auditConstant,
-            nullQueueErrorMessage, nullPlanErrorMessage);
+        nullQueueErrorMessage, nullPlanErrorMessage);
   }
 
   private void validateReservationDefinition(ReservationId reservationId,
@@ -77,17 +78,15 @@ public class ReservationInputValidator {
     String message = "";
     // check if deadline is in the past
     if (contract == null) {
-      message =
-          "Missing reservation definition."
-              + " Please try again by specifying a reservation definition.";
+      message = "Missing reservation definition."
+          + " Please try again by specifying a reservation definition.";
       RMAuditLogger.logFailure("UNKNOWN", auditConstant,
           "validate reservation input definition", "ClientRMService", message);
       throw RPCUtil.getRemoteException(message);
     }
     if (contract.getDeadline() <= clock.getTime()) {
-      message =
-          "The specified deadline: " + contract.getDeadline()
-              + " is the past. Please try again with deadline in the future.";
+      message = "The specified deadline: " + contract.getDeadline()
+          + " is the past. Please try again with deadline in the future.";
       RMAuditLogger.logFailure("UNKNOWN", auditConstant,
           "validate reservation input definition", "ClientRMService", message);
       throw RPCUtil.getRemoteException(message);
@@ -95,18 +94,16 @@ public class ReservationInputValidator {
     // Check if at least one RR has been specified
     ReservationRequests resReqs = contract.getReservationRequests();
     if (resReqs == null) {
-      message =
-          "No resources have been specified to reserve."
-              + "Please try again by specifying the resources to reserve.";
+      message = "No resources have been specified to reserve."
+          + "Please try again by specifying the resources to reserve.";
       RMAuditLogger.logFailure("UNKNOWN", auditConstant,
           "validate reservation input definition", "ClientRMService", message);
       throw RPCUtil.getRemoteException(message);
     }
     List<ReservationRequest> resReq = resReqs.getReservationResources();
     if (resReq == null || resReq.isEmpty()) {
-      message =
-          "No resources have been specified to reserve."
-              + " Please try again by specifying the resources to reserve.";
+      message = "No resources have been specified to reserve."
+          + " Please try again by specifying the resources to reserve.";
       RMAuditLogger.logFailure("UNKNOWN", auditConstant,
           "validate reservation input definition", "ClientRMService", message);
       throw RPCUtil.getRemoteException(message);
@@ -123,22 +120,18 @@ public class ReservationInputValidator {
       } else {
         minDuration += rr.getDuration();
       }
-      maxGangSize =
-          Resources.max(plan.getResourceCalculator(), plan.getTotalCapacity(),
-              maxGangSize,
-              Resources.multiply(rr.getCapability(), rr.getConcurrency()));
+      maxGangSize = Resources.max(plan.getResourceCalculator(),
+          plan.getTotalCapacity(), maxGangSize,
+          Resources.multiply(rr.getCapability(), rr.getConcurrency()));
     }
     // verify the allocation is possible (skip for ANY)
     long duration = contract.getDeadline() - contract.getArrival();
-    if (duration < minDuration
-        && type != ReservationRequestInterpreter.R_ANY) {
-      message =
-          "The time difference ("
-              + (duration)
-              + ") between arrival (" + contract.getArrival() + ") "
-              + "and deadline (" + contract.getDeadline() + ") must "
-              + " be greater or equal to the minimum resource duration ("
-              + minDuration + ")";
+    if (duration < minDuration && type != ReservationRequestInterpreter.R_ANY) {
+      message = "The time difference (" + (duration) + ") between arrival ("
+          + contract.getArrival() + ") " + "and deadline ("
+          + contract.getDeadline() + ") must "
+          + " be greater or equal to the minimum resource duration ("
+          + minDuration + ")";
       RMAuditLogger.logFailure("UNKNOWN", auditConstant,
           "validate reservation input definition", "ClientRMService", message);
       throw RPCUtil.getRemoteException(message);
@@ -148,10 +141,9 @@ public class ReservationInputValidator {
     if (Resources.greaterThan(plan.getResourceCalculator(),
         plan.getTotalCapacity(), maxGangSize, plan.getTotalCapacity())
         && type != ReservationRequestInterpreter.R_ANY) {
-      message =
-          "The size of the largest gang in the reservation definition ("
-              + maxGangSize + ") exceed the capacity available ("
-              + plan.getTotalCapacity() + " )";
+      message = "The size of the largest gang in the reservation definition ("
+          + maxGangSize + ") exceed the capacity available ("
+          + plan.getTotalCapacity() + " )";
       RMAuditLogger.logFailure("UNKNOWN", auditConstant,
           "validate reservation input definition", "ClientRMService", message);
       throw RPCUtil.getRemoteException(message);
@@ -179,32 +171,32 @@ public class ReservationInputValidator {
     }
   }
 
-  private Plan getPlanFromQueue(ReservationSystem reservationSystem, String
-          queue, String auditConstant) throws YarnException {
+  private Plan getPlanFromQueue(ReservationSystem reservationSystem,
+      String queue, String auditConstant) throws YarnException {
     String nullQueueErrorMessage = "The queue is not specified."
-            + " Please try again with a valid reservable queue.";
+        + " Please try again with a valid reservable queue.";
     String nullPlanErrorMessage = "The specified queue: " + queue
-            + " is not managed by reservation system."
-            + " Please try again with a valid reservable queue.";
+        + " is not managed by reservation system."
+        + " Please try again with a valid reservable queue.";
     return getPlanFromQueue(reservationSystem, queue, auditConstant,
-            nullQueueErrorMessage, nullPlanErrorMessage);
+        nullQueueErrorMessage, nullPlanErrorMessage);
   }
 
-  private Plan getPlanFromQueue(ReservationSystem reservationSystem, String
-          queue, String auditConstant, String nullQueueErrorMessage,
-          String nullPlanErrorMessage) throws YarnException {
+  private Plan getPlanFromQueue(ReservationSystem reservationSystem,
+      String queue, String auditConstant, String nullQueueErrorMessage,
+      String nullPlanErrorMessage) throws YarnException {
     if (queue == null || queue.isEmpty()) {
       RMAuditLogger.logFailure("UNKNOWN", auditConstant,
-              "validate reservation input", "ClientRMService",
-              nullQueueErrorMessage);
+          "validate reservation input", "ClientRMService",
+          nullQueueErrorMessage);
       throw RPCUtil.getRemoteException(nullQueueErrorMessage);
     }
     // check if the associated plan is valid
     Plan plan = reservationSystem.getPlan(queue);
     if (plan == null) {
       RMAuditLogger.logFailure("UNKNOWN", auditConstant,
-              "validate reservation input", "ClientRMService",
-              nullPlanErrorMessage);
+          "validate reservation input", "ClientRMService",
+          nullPlanErrorMessage);
       throw RPCUtil.getRemoteException(nullPlanErrorMessage);
     }
     return plan;
@@ -222,22 +214,21 @@ public class ReservationInputValidator {
    * @param reservationId the {@link ReservationId} associated with the current
    *          request
    * @return the {@link Plan} to submit the request to
-   * @throws YarnException
+   * @throws YarnException if validation fails
    */
   public Plan validateReservationSubmissionRequest(
-      ReservationSystem reservationSystem,
-      ReservationSubmissionRequest request, ReservationId reservationId)
-      throws YarnException {
+      ReservationSystem reservationSystem, ReservationSubmissionRequest request,
+      ReservationId reservationId) throws YarnException {
     String message;
     if (reservationId == null) {
-      message = "Reservation id cannot be null. Please try again " +
-        "specifying a valid reservation id by creating a new reservation id.";
+      message = "Reservation id cannot be null. Please try again specifying "
+          + " a valid reservation id by creating a new reservation id.";
       throw RPCUtil.getRemoteException(message);
     }
     // Check if it is a managed queue
     String queue = request.getQueue();
     Plan plan = getPlanFromQueue(reservationSystem, queue,
-            AuditConstants.SUBMIT_RESERVATION_REQUEST);
+        AuditConstants.SUBMIT_RESERVATION_REQUEST);
 
     validateReservationDefinition(reservationId,
         request.getReservationDefinition(), plan,
@@ -255,15 +246,14 @@ public class ReservationInputValidator {
    * @param request the {@link ReservationUpdateRequest} defining the resources
    *          required over time for the request
    * @return the {@link Plan} to submit the request to
-   * @throws YarnException
+   * @throws YarnException if validation fails
    */
   public Plan validateReservationUpdateRequest(
       ReservationSystem reservationSystem, ReservationUpdateRequest request)
       throws YarnException {
     ReservationId reservationId = request.getReservationId();
-    Plan plan =
-        validateReservation(reservationSystem, reservationId,
-            AuditConstants.UPDATE_RESERVATION_REQUEST);
+    Plan plan = validateReservation(reservationSystem, reservationId,
+        AuditConstants.UPDATE_RESERVATION_REQUEST);
     validateReservationDefinition(reservationId,
         request.getReservationDefinition(), plan,
         AuditConstants.UPDATE_RESERVATION_REQUEST);
@@ -278,28 +268,26 @@ public class ReservationInputValidator {
    *
    * @param reservationSystem the {@link ReservationSystem} to validate against
    * @param request the {@link ReservationListRequest} defining search
-   *                parameters for reservations in the {@link ReservationSystem}
-   *                that is being validated against.
+   *          parameters for reservations in the {@link ReservationSystem} that
+   *          is being validated against.
    * @return the {@link Plan} to list reservations of.
-   * @throws YarnException
+   * @throws YarnException if validation fails
    */
   public Plan validateReservationListRequest(
-      ReservationSystem reservationSystem,
-      ReservationListRequest request)
+      ReservationSystem reservationSystem, ReservationListRequest request)
       throws YarnException {
     String queue = request.getQueue();
     if (request.getEndTime() < request.getStartTime()) {
-      String errorMessage = "The specified end time must be greater than " +
-              "the specified start time.";
+      String errorMessage = "The specified end time must be greater than "
+          + "the specified start time.";
       RMAuditLogger.logFailure("UNKNOWN",
-              AuditConstants.LIST_RESERVATION_REQUEST,
-              "validate list reservation input", "ClientRMService",
-              errorMessage);
+          AuditConstants.LIST_RESERVATION_REQUEST,
+          "validate list reservation input", "ClientRMService", errorMessage);
       throw RPCUtil.getRemoteException(errorMessage);
     }
     // Check if it is a managed queue
     return getPlanFromQueue(reservationSystem, queue,
-            AuditConstants.LIST_RESERVATION_REQUEST);
+        AuditConstants.LIST_RESERVATION_REQUEST);
   }
 
   /**
@@ -312,7 +300,7 @@ public class ReservationInputValidator {
    * @param request the {@link ReservationDeleteRequest} defining the resources
    *          required over time for the request
    * @return the {@link Plan} to submit the request to
-   * @throws YarnException
+   * @throws YarnException if validation fails
    */
   public Plan validateReservationDeleteRequest(
       ReservationSystem reservationSystem, ReservationDeleteRequest request)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7996eca7/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/ReservationSystem.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/reservation/ReservationSystem.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/ReservationSystem.java
index 8b62972..a6c8fcf 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/ReservationSystem.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/ReservationSystem.java
@@ -18,6 +18,8 @@
 
 package org.apache.hadoop.yarn.server.resourcemanager.reservation;
 
+import java.util.Map;
+
 import org.apache.hadoop.classification.InterfaceAudience.LimitedPrivate;
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.conf.Configuration;
@@ -29,8 +31,6 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Queue;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.security.ReservationsACLsManager;
 
-import java.util.Map;
-
 /**
  * This interface is the one implemented by any system that wants to support
  * Reservations i.e. make {@code Resource} allocations in future. Implementors
@@ -57,7 +57,7 @@ public interface ReservationSystem extends Recoverable {
    * 
    * @param conf configuration
    * @param rmContext current context of the {@code ResourceManager}
-   * @throws YarnException
+   * @throws YarnException if initialization of the configured plan fails
    */
   void reinitialize(Configuration conf, RMContext rmContext)
       throws YarnException;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7996eca7/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/SharingPolicy.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/reservation/SharingPolicy.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/SharingPolicy.java
index e458055..cbf0f38 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/SharingPolicy.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/SharingPolicy.java
@@ -38,7 +38,7 @@ public interface SharingPolicy {
    * @param planQueuePath the name of the queue for this plan
    * @param conf the system configuration
    */
-  public void init(String planQueuePath, ReservationSchedulerConfiguration conf);
+  void init(String planQueuePath, ReservationSchedulerConfiguration conf);
 
   /**
    * This method runs the policy validation logic, and return true/false on
@@ -51,7 +51,7 @@ public interface SharingPolicy {
    * @throws PlanningException if the policy is respected if we add this
    *           {@link ReservationAllocation} to the {@link Plan}
    */
-  public void validate(Plan plan, ReservationAllocation newAllocation)
+  void validate(Plan plan, ReservationAllocation newAllocation)
       throws PlanningException;
 
   /**
@@ -68,9 +68,13 @@ public interface SharingPolicy {
    * @param start the start time for the range we are querying
    * @param end the end time for the range we are querying
    * @param oldId (optional) the id of a reservation being updated
+   *
+   * @return the available resources expressed as a
+   *         {@link RLESparseResourceAllocation}
+   *
    * @throws PlanningException throws if the request is not valid
    */
-  public RLESparseResourceAllocation availableResources(
+  RLESparseResourceAllocation availableResources(
       RLESparseResourceAllocation available, Plan plan, String user,
       ReservationId oldId, long start, long end) throws PlanningException;
 
@@ -82,7 +86,6 @@ public interface SharingPolicy {
    * 
    * @return validWindow the window of validity considered by the policy.
    */
-  public long getValidWindow();
-
+  long getValidWindow();
 
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7996eca7/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/planning/Planner.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/reservation/planning/Planner.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/planning/Planner.java
index abac6ac..af0e712 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/planning/Planner.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/planning/Planner.java
@@ -34,7 +34,7 @@ public interface Planner {
    *
    * @param plan the {@link Plan} to replan
    * @param contracts the list of reservation requests
-   * @throws PlanningException
+   * @throws PlanningException if operation is unsuccessful
    */
   public void plan(Plan plan, List<ReservationDefinition> contracts)
       throws PlanningException;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7996eca7/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/planning/PlanningAlgorithm.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/reservation/planning/PlanningAlgorithm.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/planning/PlanningAlgorithm.java
index 199bfa5..bbbf0d6 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/planning/PlanningAlgorithm.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/planning/PlanningAlgorithm.java
@@ -50,7 +50,7 @@ public abstract class PlanningAlgorithm implements ReservationAgent {
    * @return whether the allocateUser function was successful or not
    *
    * @throws PlanningException if the session cannot be fitted into the plan
-   * @throws ContractValidationException
+   * @throws ContractValidationException if validation fails
    */
   protected boolean allocateUser(ReservationId reservationId, String user,
       Plan plan, ReservationDefinition contract,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7996eca7/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/planning/StageAllocator.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/reservation/planning/StageAllocator.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/planning/StageAllocator.java
index ec6d9c0..8934b0f 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/planning/StageAllocator.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/planning/StageAllocator.java
@@ -50,7 +50,7 @@ public interface StageAllocator {
    *
    * @return The computed allocation (or null if the stage could not be
    *         allocated)
-   * @throws PlanningException
+   * @throws PlanningException if operation is unsuccessful
    */
   Map<ReservationInterval, Resource> computeStageAllocation(Plan plan,
       RLESparseResourceAllocation planLoads,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7996eca7/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/planning/StageAllocatorGreedy.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/reservation/planning/StageAllocatorGreedy.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/planning/StageAllocatorGreedy.java
index da04336..d107487 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/planning/StageAllocatorGreedy.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/planning/StageAllocatorGreedy.java
@@ -69,7 +69,7 @@ public class StageAllocatorGreedy implements StageAllocator {
 
     RLESparseResourceAllocation netAvailable =
         plan.getAvailableResourceOverTime(user, oldId, stageEarliestStart,
-            stageDeadline);
+            stageDeadline, 0);
 
     netAvailable =
         RLESparseResourceAllocation.merge(plan.getResourceCalculator(),

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7996eca7/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/planning/StageAllocatorGreedyRLE.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/reservation/planning/StageAllocatorGreedyRLE.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/planning/StageAllocatorGreedyRLE.java
index ec83e02..ae7d91a 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/planning/StageAllocatorGreedyRLE.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/planning/StageAllocatorGreedyRLE.java
@@ -83,9 +83,8 @@ public class StageAllocatorGreedyRLE implements StageAllocator {
     int gangsToPlace = rr.getNumContainers() / rr.getConcurrency();
 
     // get available resources from plan
-    RLESparseResourceAllocation netRLERes =
-        plan.getAvailableResourceOverTime(user, oldId, stageEarliestStart,
-            stageDeadline);
+    RLESparseResourceAllocation netRLERes = plan.getAvailableResourceOverTime(
+        user, oldId, stageEarliestStart, stageDeadline, 0);
 
     // remove plan modifications
     netRLERes =

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7996eca7/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/planning/StageAllocatorLowCostAligned.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/reservation/planning/StageAllocatorLowCostAligned.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/planning/StageAllocatorLowCostAligned.java
index e45f58c..c014549 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/planning/StageAllocatorLowCostAligned.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/planning/StageAllocatorLowCostAligned.java
@@ -77,8 +77,8 @@ public class StageAllocatorLowCostAligned implements StageAllocator {
     ResourceCalculator resCalc = plan.getResourceCalculator();
     Resource capacity = plan.getTotalCapacity();
 
-    RLESparseResourceAllocation netRLERes = plan
-        .getAvailableResourceOverTime(user, oldId, stageArrival, stageDeadline);
+    RLESparseResourceAllocation netRLERes = plan.getAvailableResourceOverTime(
+        user, oldId, stageArrival, stageDeadline, 0);
 
     long step = plan.getStep();
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7996eca7/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/ReservationSystemTestUtil.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/reservation/ReservationSystemTestUtil.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/ReservationSystemTestUtil.java
index e99842e..5337e06 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/ReservationSystemTestUtil.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/ReservationSystemTestUtil.java
@@ -19,7 +19,10 @@ package org.apache.hadoop.yarn.server.resourcemanager.reservation;
 
 import static org.mockito.Matchers.any;
 import static org.mockito.Matchers.anySetOf;
-import static org.mockito.Mockito.*;
+import static org.mockito.Mockito.doReturn;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.spy;
+import static org.mockito.Mockito.when;
 
 import java.io.FileWriter;
 import java.io.IOException;
@@ -76,7 +79,8 @@ public class ReservationSystemTestUtil {
       String reservationQ, long timeWindow, float instConstraint,
       float avgConstraint) {
 
-    ReservationSchedulerConfiguration realConf = new CapacitySchedulerConfiguration();
+    ReservationSchedulerConfiguration realConf =
+        new CapacitySchedulerConfiguration();
     ReservationSchedulerConfiguration conf = spy(realConf);
     when(conf.getReservationWindow(reservationQ)).thenReturn(timeWindow);
     when(conf.getInstantaneousMaxCapacity(reservationQ))
@@ -168,7 +172,6 @@ public class ReservationSystemTestUtil {
     scheduler.start();
     scheduler.reinitialize(conf, rmContext);
 
-
     Resource resource =
         ReservationSystemTestUtil.calculateClusterResource(numContainers);
     RMNode node1 = MockNodes.newNodeInfo(1, resource, 1, "127.0.0.1");
@@ -184,10 +187,16 @@ public class ReservationSystemTestUtil {
 
   public static ReservationDefinition createSimpleReservationDefinition(
       long arrival, long deadline, long duration, int parallelism) {
+    return createSimpleReservationDefinition(arrival, deadline, duration,
+        parallelism, null);
+  }
+
+  public static ReservationDefinition createSimpleReservationDefinition(
+      long arrival, long deadline, long duration, int parallelism,
+      String recurrenceExpression) {
     // create a request with a single atomic ask
-    ReservationRequest r =
-        ReservationRequest.newInstance(Resource.newInstance(1024, 1),
-            parallelism, parallelism, duration);
+    ReservationRequest r = ReservationRequest.newInstance(
+        Resource.newInstance(1024, 1), parallelism, parallelism, duration);
     ReservationDefinition rDef = new ReservationDefinitionPBImpl();
     ReservationRequests reqs = new ReservationRequestsPBImpl();
     reqs.setReservationResources(Collections.singletonList(r));
@@ -195,32 +204,31 @@ public class ReservationSystemTestUtil {
     rDef.setReservationRequests(reqs);
     rDef.setArrival(arrival);
     rDef.setDeadline(deadline);
+    if (recurrenceExpression != null) {
+      rDef.setRecurrenceExpression(recurrenceExpression);
+    }
     return rDef;
   }
 
   public static ReservationSubmissionRequest createSimpleReservationRequest(
       ReservationId reservationId, int numContainers, long arrival,
       long deadline, long duration) {
-    return createSimpleReservationRequest(reservationId, numContainers,
-        arrival, deadline, duration, Priority.UNDEFINED);
+    return createSimpleReservationRequest(reservationId, numContainers, arrival,
+        deadline, duration, Priority.UNDEFINED);
   }
 
   public static ReservationSubmissionRequest createSimpleReservationRequest(
       ReservationId reservationId, int numContainers, long arrival,
       long deadline, long duration, Priority priority) {
     // create a request with a single atomic ask
-    ReservationRequest r =
-        ReservationRequest.newInstance(Resource.newInstance(1024, 1),
-            numContainers, 1, duration);
-    ReservationRequests reqs =
-        ReservationRequests.newInstance(Collections.singletonList(r),
-            ReservationRequestInterpreter.R_ALL);
-    ReservationDefinition rDef =
-        ReservationDefinition.newInstance(arrival, deadline, reqs,
-            "testClientRMService#reservation", "0", priority);
-    ReservationSubmissionRequest request =
-        ReservationSubmissionRequest.newInstance(rDef,
-            reservationQ, reservationId);
+    ReservationRequest r = ReservationRequest
+        .newInstance(Resource.newInstance(1024, 1), numContainers, 1, duration);
+    ReservationRequests reqs = ReservationRequests.newInstance(
+        Collections.singletonList(r), ReservationRequestInterpreter.R_ALL);
+    ReservationDefinition rDef = ReservationDefinition.newInstance(arrival,
+        deadline, reqs, "testClientRMService#reservation", "0", priority);
+    ReservationSubmissionRequest request = ReservationSubmissionRequest
+        .newInstance(rDef, reservationQ, reservationId);
     return request;
   }
 
@@ -252,9 +260,9 @@ public class ReservationSystemTestUtil {
     return cs;
   }
 
-  @SuppressWarnings("rawtypes") public static void initializeRMContext(
-      int numContainers, AbstractYarnScheduler scheduler,
-      RMContext mockRMContext) {
+  @SuppressWarnings("rawtypes")
+  public static void initializeRMContext(int numContainers,
+      AbstractYarnScheduler scheduler, RMContext mockRMContext) {
 
     when(mockRMContext.getScheduler()).thenReturn(scheduler);
     Resource r = calculateClusterResource(numContainers);
@@ -262,26 +270,25 @@ public class ReservationSystemTestUtil {
   }
 
   public static RMContext createRMContext(Configuration conf) {
-    RMContext mockRmContext = Mockito.spy(
-        new RMContextImpl(null, null, null, null, null, null,
-            new RMContainerTokenSecretManager(conf),
-            new NMTokenSecretManagerInRM(conf),
-            new ClientToAMTokenSecretManagerInRM(), null));
+    RMContext mockRmContext = Mockito.spy(new RMContextImpl(null, null, null,
+        null, null, null, new RMContainerTokenSecretManager(conf),
+        new NMTokenSecretManagerInRM(conf),
+        new ClientToAMTokenSecretManagerInRM(), null));
 
     RMNodeLabelsManager nlm = mock(RMNodeLabelsManager.class);
     when(nlm.getQueueResource(any(String.class), anySetOf(String.class),
-            any(Resource.class))).thenAnswer(new Answer<Resource>() {
-      @Override public Resource answer(InvocationOnMock invocation)
-          throws Throwable {
-        Object[] args = invocation.getArguments();
-        return (Resource) args[2];
-      }
-    });
+        any(Resource.class))).thenAnswer(new Answer<Resource>() {
+          @Override
+          public Resource answer(InvocationOnMock invocation) throws Throwable {
+            Object[] args = invocation.getArguments();
+            return (Resource) args[2];
+          }
+        });
 
     when(nlm.getResourceByLabel(any(String.class), any(Resource.class)))
         .thenAnswer(new Answer<Resource>() {
-          @Override public Resource answer(InvocationOnMock invocation)
-              throws Throwable {
+          @Override
+          public Resource answer(InvocationOnMock invocation) throws Throwable {
             Object[] args = invocation.getArguments();
             return (Resource) args[1];
           }
@@ -304,9 +311,8 @@ public class ReservationSystemTestUtil {
     final String A = CapacitySchedulerConfiguration.ROOT + ".a";
     conf.setCapacity(A, 10);
 
-    final String dedicated =
-        CapacitySchedulerConfiguration.ROOT + CapacitySchedulerConfiguration.DOT
-            + reservationQ;
+    final String dedicated = CapacitySchedulerConfiguration.ROOT
+        + CapacitySchedulerConfiguration.DOT + reservationQ;
     conf.setCapacity(dedicated, 80);
     // Set as reservation queue
     conf.setReservable(dedicated, true);
@@ -405,26 +411,55 @@ public class ReservationSystemTestUtil {
 
   public static Map<ReservationInterval, Resource> generateAllocation(
       long startTime, long step, int[] alloc) {
+    return generateAllocation(startTime, step, alloc, null);
+  }
+
+  public static Map<ReservationInterval, Resource> generateAllocation(
+      long startTime, long step, int[] alloc, String recurrenceExpression) {
     Map<ReservationInterval, Resource> req = new TreeMap<>();
-    for (int i = 0; i < alloc.length; i++) {
-      req.put(new ReservationInterval(startTime + i * step,
-          startTime + (i + 1) * step), ReservationSystemUtil.toResource(
-          ReservationRequest
-              .newInstance(Resource.newInstance(1024, 1), alloc[i])));
+
+    long period = 0;
+    if (recurrenceExpression != null) {
+      period = Long.parseLong(recurrenceExpression);
+    }
+
+    long rStart;
+    long rEnd;
+    for (int j = 0; j < 86400000; j += period) {
+      for (int i = 0; i < alloc.length; i++) {
+        rStart = (startTime + i * step) + j * period;
+        rEnd = (startTime + (i + 1) * step) + j * period;
+        if (period > 0) {
+          rStart = rStart % period + j * period;
+          rEnd = rEnd % period + j * period;
+          if (rStart > rEnd) {
+            // skip wrap-around entry
+            continue;
+          }
+        }
+
+        req.put(new ReservationInterval(rStart, rEnd),
+            ReservationSystemUtil.toResource(ReservationRequest
+                .newInstance(Resource.newInstance(1024, 1), alloc[i])));
+
+      }
+      // execute only once if non-periodic
+      if (period == 0) {
+        break;
+      }
     }
     return req;
   }
 
-  public static RLESparseResourceAllocation
-      generateRLESparseResourceAllocation(int[] alloc, long[] timeSteps) {
+  public static RLESparseResourceAllocation generateRLESparseResourceAllocation(
+      int[] alloc, long[] timeSteps) {
     TreeMap<Long, Resource> allocationsMap = new TreeMap<>();
     for (int i = 0; i < alloc.length; i++) {
       allocationsMap.put(timeSteps[i],
           Resource.newInstance(alloc[i], alloc[i]));
     }
-    RLESparseResourceAllocation rleVector =
-        new RLESparseResourceAllocation(allocationsMap,
-            new DefaultResourceCalculator());
+    RLESparseResourceAllocation rleVector = new RLESparseResourceAllocation(
+        allocationsMap, new DefaultResourceCalculator());
     return rleVector;
   }
 


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[28/37] hadoop git commit: YARN-6992. Kill application button is visible even if the application is FINISHED in RM UI. Contributed by Suma Shivaprasad.

Posted by ae...@apache.org.
YARN-6992. Kill application button is visible even if the application is FINISHED in RM UI. Contributed by Suma Shivaprasad.


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

Branch: refs/heads/HDFS-7240
Commit: 6f101e7df10d477cf7469f6979d0cb8269315a3e
Parents: b6e7d13
Author: Sunil G <su...@apache.org>
Authored: Thu Sep 7 13:16:20 2017 +0530
Committer: Sunil G <su...@apache.org>
Committed: Thu Sep 7 13:16:20 2017 +0530

----------------------------------------------------------------------
 .../java/org/apache/hadoop/yarn/server/webapp/AppBlock.java | 9 ++++++++-
 1 file changed, 8 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/6f101e7d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/AppBlock.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/AppBlock.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/AppBlock.java
index 693aa04..95bc0aa 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/AppBlock.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/AppBlock.java
@@ -144,7 +144,8 @@ public class AppBlock extends HtmlBlock {
         && webUiType.equals(YarnWebParams.RM_WEB_UI)
         && conf.getBoolean(YarnConfiguration.RM_WEBAPP_UI_ACTIONS_ENABLED,
           YarnConfiguration.DEFAULT_RM_WEBAPP_UI_ACTIONS_ENABLED)
-            && !unsecuredUIForSecuredCluster) {
+            && !unsecuredUIForSecuredCluster
+            && !isAppInFinalState(app)) {
       // Application Kill
       html.div()
         .button()
@@ -419,4 +420,10 @@ public class AppBlock extends HtmlBlock {
     }
     return ret;
   }
+
+  private boolean isAppInFinalState(AppInfo app) {
+    return app.getAppState() == YarnApplicationState.FINISHED
+        || app.getAppState() == YarnApplicationState.FAILED
+        || app.getAppState() == YarnApplicationState.KILLED;
+  }
 }


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[25/37] hadoop git commit: YARN-5328. Plan/ResourceAllocation data structure enhancements required to support recurring reservations in ReservationSystem.

Posted by ae...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/b6e7d136/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestInMemoryPlan.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/reservation/TestInMemoryPlan.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestInMemoryPlan.java
index bc98e2f..03569d4 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestInMemoryPlan.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestInMemoryPlan.java
@@ -40,6 +40,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.reservation.planning.Planne
 import org.apache.hadoop.yarn.server.resourcemanager.reservation.planning.ReservationAgent;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueMetrics;
 import org.apache.hadoop.yarn.util.Clock;
+import org.apache.hadoop.yarn.util.UTCClock;
 import org.apache.hadoop.yarn.util.resource.DefaultResourceCalculator;
 import org.apache.hadoop.yarn.util.resource.ResourceCalculator;
 import org.junit.After;
@@ -47,6 +48,10 @@ import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
 
+/**
+ * Testing the class {@link InMemoryPlan}.
+ */
+@SuppressWarnings("checkstyle:nowhitespaceafter")
 public class TestInMemoryPlan {
 
   private String user = "yarn";
@@ -62,6 +67,7 @@ public class TestInMemoryPlan {
   private ReservationAgent agent;
   private Planner replanner;
   private RMContext context;
+  private long maxPeriodicity;
 
   @Before
   public void setUp() throws PlanningException {
@@ -72,7 +78,7 @@ public class TestInMemoryPlan {
 
     clock = mock(Clock.class);
     queueMetrics = mock(QueueMetrics.class);
-    policy = mock(SharingPolicy.class);
+    policy = new NoOverCommitPolicy();
     replanner = mock(Planner.class);
 
     when(clock.getTime()).thenReturn(1L);
@@ -95,15 +101,41 @@ public class TestInMemoryPlan {
 
   @Test
   public void testAddReservation() {
-    Plan plan =
-        new InMemoryPlan(queueMetrics, policy, agent, totalCapacity, 1L,
-            resCalc, minAlloc, maxAlloc, planName, replanner, true, context);
+    Plan plan = new InMemoryPlan(queueMetrics, policy, agent, totalCapacity, 1L,
+        resCalc, minAlloc, maxAlloc, planName, replanner, true, context);
     ReservationId reservationID =
-            ReservationSystemTestUtil.getNewReservationId();
+        ReservationSystemTestUtil.getNewReservationId();
     int[] alloc = { 10, 10, 10, 10, 10, 10 };
     int start = 100;
-    ReservationAllocation rAllocation = createReservationAllocation
-            (reservationID, start, alloc);
+    ReservationAllocation rAllocation =
+        createReservationAllocation(reservationID, start, alloc);
+    Assert.assertNull(plan.getReservationById(reservationID));
+    try {
+      plan.addReservation(rAllocation, false);
+    } catch (PlanningException e) {
+      Assert.fail(e.getMessage());
+    }
+    doAssertions(plan, rAllocation);
+    checkAllocation(plan, alloc, start, 0);
+  }
+
+  @Test
+  public void testAddPeriodicReservation() throws PlanningException {
+
+    maxPeriodicity = 100;
+    Plan plan = new InMemoryPlan(queueMetrics, policy, agent, totalCapacity, 1L,
+        resCalc, minAlloc, maxAlloc, planName, replanner, true, maxPeriodicity,
+        context, new UTCClock());
+
+    ReservationId reservationID =
+        ReservationSystemTestUtil.getNewReservationId();
+    int[] alloc = { 10, 50 };
+    int start = 10;
+    long period = 20;
+    ReservationAllocation rAllocation = createReservationAllocation(
+        reservationID, start, alloc, String.valueOf(period));
+    // use periodicity of 1hr
+    rAllocation.setPeriodicity(period);
     Assert.assertNull(plan.getReservationById(reservationID));
     try {
       plan.addReservation(rAllocation, false);
@@ -111,32 +143,54 @@ public class TestInMemoryPlan {
       Assert.fail(e.getMessage());
     }
     doAssertions(plan, rAllocation);
-    checkAllocation(plan, alloc, start);
+    checkAllocation(plan, alloc, start, period);
+
+    RLESparseResourceAllocation available =
+        plan.getAvailableResourceOverTime(user, reservationID, 150, 330, 50);
+    System.out.println(available);
   }
 
-  private void checkAllocation(Plan plan, int[] alloc, int start) {
+  private void checkAllocation(Plan plan, int[] alloc, int start,
+      long periodicity) {
+    long end = start + alloc.length;
+    if (periodicity > 0) {
+      end = end + maxPeriodicity;
+    }
     RLESparseResourceAllocation userCons =
-        plan.getConsumptionForUserOverTime(user, start, start + alloc.length);
+        plan.getConsumptionForUserOverTime(user, start, end * 3);
 
     for (int i = 0; i < alloc.length; i++) {
-      Assert.assertEquals(Resource.newInstance(1024 * (alloc[i]), (alloc[i])),
-          plan.getTotalCommittedResources(start + i));
-      Assert.assertEquals(Resource.newInstance(1024 * (alloc[i]), (alloc[i])),
-          userCons.getCapacityAtTime(start + i));
+      // only one instance for non-periodic reservation
+      if (periodicity <= 0) {
+        Assert.assertEquals(Resource.newInstance(1024 * (alloc[i]), (alloc[i])),
+            plan.getTotalCommittedResources(start + i));
+        Assert.assertEquals(Resource.newInstance(1024 * (alloc[i]), (alloc[i])),
+            userCons.getCapacityAtTime(start + i));
+      } else {
+        // periodic reservations should repeat
+        long y = 0;
+        Resource res = Resource.newInstance(1024 * (alloc[i]), (alloc[i]));
+        while (y <= end * 2) {
+          Assert.assertEquals("At time: " + start + i + y, res,
+              plan.getTotalCommittedResources(start + i + y));
+          Assert.assertEquals(" At time: " + (start + i + y), res,
+              userCons.getCapacityAtTime(start + i + y));
+          y = y + periodicity;
+        }
+      }
     }
   }
 
   @Test
   public void testAddEmptyReservation() {
-    Plan plan =
-        new InMemoryPlan(queueMetrics, policy, agent, totalCapacity, 1L,
-            resCalc, minAlloc, maxAlloc, planName, replanner, true, context);
+    Plan plan = new InMemoryPlan(queueMetrics, policy, agent, totalCapacity, 1L,
+        resCalc, minAlloc, maxAlloc, planName, replanner, true, context);
     ReservationId reservationID =
-            ReservationSystemTestUtil.getNewReservationId();
+        ReservationSystemTestUtil.getNewReservationId();
     int[] alloc = {};
     int start = 100;
-    ReservationAllocation rAllocation = createReservationAllocation
-            (reservationID, start, alloc);
+    ReservationAllocation rAllocation =
+        createReservationAllocation(reservationID, start, alloc);
     Assert.assertNull(plan.getReservationById(reservationID));
     try {
       plan.addReservation(rAllocation, false);
@@ -148,15 +202,14 @@ public class TestInMemoryPlan {
   @Test
   public void testAddReservationAlreadyExists() {
     // First add a reservation
-    Plan plan =
-        new InMemoryPlan(queueMetrics, policy, agent, totalCapacity, 1L,
-            resCalc, minAlloc, maxAlloc, planName, replanner, true, context);
+    Plan plan = new InMemoryPlan(queueMetrics, policy, agent, totalCapacity, 1L,
+        resCalc, minAlloc, maxAlloc, planName, replanner, true, context);
     ReservationId reservationID =
-            ReservationSystemTestUtil.getNewReservationId();
+        ReservationSystemTestUtil.getNewReservationId();
     int[] alloc = { 10, 10, 10, 10, 10, 10 };
     int start = 100;
-    ReservationAllocation rAllocation = createReservationAllocation
-            (reservationID, start, alloc);
+    ReservationAllocation rAllocation =
+        createReservationAllocation(reservationID, start, alloc);
     Assert.assertNull(plan.getReservationById(reservationID));
     try {
       plan.addReservation(rAllocation, false);
@@ -164,7 +217,7 @@ public class TestInMemoryPlan {
       Assert.fail(e.getMessage());
     }
     doAssertions(plan, rAllocation);
-    checkAllocation(plan, alloc, start);
+    checkAllocation(plan, alloc, start, 0);
 
     // Try to add it again
     try {
@@ -180,16 +233,15 @@ public class TestInMemoryPlan {
 
   @Test
   public void testUpdateReservation() {
-    Plan plan =
-        new InMemoryPlan(queueMetrics, policy, agent, totalCapacity, 1L,
-            resCalc, minAlloc, maxAlloc, planName, replanner, true, context);
+    Plan plan = new InMemoryPlan(queueMetrics, policy, agent, totalCapacity, 1L,
+        resCalc, minAlloc, maxAlloc, planName, replanner, true, context);
     ReservationId reservationID =
         ReservationSystemTestUtil.getNewReservationId();
     // First add a reservation
     int[] alloc = { 10, 10, 10, 10, 10, 10 };
     int start = 100;
-    ReservationAllocation rAllocation = createReservationAllocation
-            (reservationID, start, alloc);
+    ReservationAllocation rAllocation =
+        createReservationAllocation(reservationID, start, alloc);
     Assert.assertNull(plan.getReservationById(reservationID));
     try {
       plan.addReservation(rAllocation, false);
@@ -210,8 +262,8 @@ public class TestInMemoryPlan {
     // Now update it
     start = 110;
     int[] updatedAlloc = { 0, 5, 10, 10, 5, 0 };
-    rAllocation = createReservationAllocation(reservationID, start,
-            updatedAlloc, true);
+    rAllocation =
+        createReservationAllocation(reservationID, start, updatedAlloc, true);
     try {
       plan.updateReservation(rAllocation);
     } catch (PlanningException e) {
@@ -219,32 +271,71 @@ public class TestInMemoryPlan {
     }
     doAssertions(plan, rAllocation);
 
-    userCons =
-        plan.getConsumptionForUserOverTime(user, start, start
-            + updatedAlloc.length);
+    userCons = plan.getConsumptionForUserOverTime(user, start,
+        start + updatedAlloc.length);
 
     for (int i = 0; i < updatedAlloc.length; i++) {
-      Assert.assertEquals(
-     Resource.newInstance(1024 * (updatedAlloc[i] + i), updatedAlloc[i]
-              + i), plan.getTotalCommittedResources(start + i));
-      Assert.assertEquals(
-          Resource.newInstance(1024 * (updatedAlloc[i] + i), updatedAlloc[i]
-              + i), userCons.getCapacityAtTime(start + i));
+      Assert.assertEquals(Resource.newInstance(1024 * (updatedAlloc[i] + i),
+          updatedAlloc[i] + i), plan.getTotalCommittedResources(start + i));
+      Assert.assertEquals(Resource.newInstance(1024 * (updatedAlloc[i] + i),
+          updatedAlloc[i] + i), userCons.getCapacityAtTime(start + i));
+    }
+  }
+
+  @Test
+  public void testUpdatePeriodicReservation() {
+    Plan plan = new InMemoryPlan(queueMetrics, policy, agent, totalCapacity, 1L,
+        resCalc, minAlloc, maxAlloc, planName, replanner, true, context);
+    // First add a reservation
+    ReservationId reservationID =
+        ReservationSystemTestUtil.getNewReservationId();
+    int[] alloc = { 10, 20 };
+    int start = 100;
+    ReservationAllocation rAllocation =
+        createReservationAllocation(reservationID, start, alloc);
+    // use periodicity of 1hr
+    long period = 3600000;
+    rAllocation.getReservationDefinition()
+        .setRecurrenceExpression(String.valueOf(period));
+    rAllocation.setPeriodicity(period);
+    Assert.assertNull(plan.getReservationById(reservationID));
+    try {
+      plan.addReservation(rAllocation, false);
+    } catch (PlanningException e) {
+      Assert.fail(e.getMessage());
+    }
+    System.out.println(plan.toString());
+    doAssertions(plan, rAllocation);
+    checkAllocation(plan, alloc, start, period);
+
+    // Now update it
+    start = 110;
+    int[] updatedAlloc = { 30, 40, 50 };
+    rAllocation =
+        createReservationAllocation(reservationID, start, updatedAlloc);
+    rAllocation.getReservationDefinition()
+        .setRecurrenceExpression(String.valueOf(period));
+    rAllocation.setPeriodicity(period);
+    try {
+      plan.updateReservation(rAllocation);
+    } catch (PlanningException e) {
+      Assert.fail(e.getMessage());
     }
+    doAssertions(plan, rAllocation);
+    checkAllocation(plan, updatedAlloc, start, period);
   }
 
   @Test
   public void testUpdateNonExistingReservation() {
-    Plan plan =
-        new InMemoryPlan(queueMetrics, policy, agent, totalCapacity, 1L,
-            resCalc, minAlloc, maxAlloc, planName, replanner, true, context);
+    Plan plan = new InMemoryPlan(queueMetrics, policy, agent, totalCapacity, 1L,
+        resCalc, minAlloc, maxAlloc, planName, replanner, true, context);
     ReservationId reservationID =
         ReservationSystemTestUtil.getNewReservationId();
     // Try to update a reservation without adding
     int[] alloc = { 10, 10, 10, 10, 10, 10 };
     int start = 100;
     ReservationAllocation rAllocation =
-            createReservationAllocation(reservationID, start, alloc);
+        createReservationAllocation(reservationID, start, alloc);
     Assert.assertNull(plan.getReservationById(reservationID));
     try {
       plan.updateReservation(rAllocation);
@@ -260,15 +351,14 @@ public class TestInMemoryPlan {
   @Test
   public void testDeleteReservation() {
     // First add a reservation
-    Plan plan =
-        new InMemoryPlan(queueMetrics, policy, agent, totalCapacity, 1L,
-            resCalc, minAlloc, maxAlloc, planName, replanner, true, context);
+    Plan plan = new InMemoryPlan(queueMetrics, policy, agent, totalCapacity, 1L,
+        resCalc, minAlloc, maxAlloc, planName, replanner, true, context);
     ReservationId reservationID =
         ReservationSystemTestUtil.getNewReservationId();
     int[] alloc = { 10, 10, 10, 10, 10, 10 };
     int start = 100;
     ReservationAllocation rAllocation =
-            createReservationAllocation(reservationID, start, alloc, true);
+        createReservationAllocation(reservationID, start, alloc, true);
     Assert.assertNull(plan.getReservationById(reservationID));
     try {
       plan.addReservation(rAllocation, false);
@@ -307,10 +397,46 @@ public class TestInMemoryPlan {
   }
 
   @Test
+  public void testDeletePeriodicReservation() {
+    Plan plan = new InMemoryPlan(queueMetrics, policy, agent, totalCapacity, 1L,
+        resCalc, minAlloc, maxAlloc, planName, replanner, true, context);
+    // First add a reservation
+    ReservationId reservationID =
+        ReservationSystemTestUtil.getNewReservationId();
+    int[] alloc = { 10, 20 };
+    int start = 100;
+    ReservationAllocation rAllocation =
+        createReservationAllocation(reservationID, start, alloc);
+    // use periodicity of 1hr
+    long period = 3600000;
+    rAllocation.getReservationDefinition()
+        .setRecurrenceExpression(String.valueOf(period));
+    rAllocation.setPeriodicity(period);
+    Assert.assertNull(plan.getReservationById(reservationID));
+    try {
+      plan.addReservation(rAllocation, false);
+    } catch (PlanningException e) {
+      Assert.fail(e.getMessage());
+    }
+    System.out.println(plan.toString());
+    doAssertions(plan, rAllocation);
+    checkAllocation(plan, alloc, start, period);
+
+    // Now delete it
+    try {
+      plan.deleteReservation(reservationID);
+    } catch (PlanningException e) {
+      Assert.fail(e.getMessage());
+    }
+    Assert.assertNull(plan.getReservationById(reservationID));
+    System.out.print(plan);
+    checkAllocation(plan, new int[] { 0, 0 }, start, period);
+  }
+
+  @Test
   public void testDeleteNonExistingReservation() {
-    Plan plan =
-        new InMemoryPlan(queueMetrics, policy, agent, totalCapacity, 1L,
-            resCalc, minAlloc, maxAlloc, planName, replanner, true, context);
+    Plan plan = new InMemoryPlan(queueMetrics, policy, agent, totalCapacity, 1L,
+        resCalc, minAlloc, maxAlloc, planName, replanner, true, context);
     ReservationId reservationID =
         ReservationSystemTestUtil.getNewReservationId();
     // Try to delete a reservation without adding
@@ -328,8 +454,9 @@ public class TestInMemoryPlan {
 
   @Test
   public void testArchiveCompletedReservations() {
+    SharingPolicy sharingPolicy = mock(SharingPolicy.class);
     Plan plan =
-        new InMemoryPlan(queueMetrics, policy, agent, totalCapacity, 1L,
+        new InMemoryPlan(queueMetrics, sharingPolicy, agent, totalCapacity, 1L,
             resCalc, minAlloc, maxAlloc, planName, replanner, true, context);
     ReservationId reservationID1 =
         ReservationSystemTestUtil.getNewReservationId();
@@ -337,7 +464,7 @@ public class TestInMemoryPlan {
     int[] alloc1 = { 10, 10, 10, 10, 10, 10 };
     int start = 100;
     ReservationAllocation rAllocation =
-            createReservationAllocation(reservationID1, start, alloc1);
+        createReservationAllocation(reservationID1, start, alloc1);
     Assert.assertNull(plan.getReservationById(reservationID1));
     try {
       plan.addReservation(rAllocation, false);
@@ -345,15 +472,14 @@ public class TestInMemoryPlan {
       Assert.fail(e.getMessage());
     }
     doAssertions(plan, rAllocation);
-    checkAllocation(plan, alloc1, start);
-
+    checkAllocation(plan, alloc1, start, 0);
 
     // Now add another one
     ReservationId reservationID2 =
         ReservationSystemTestUtil.getNewReservationId();
     int[] alloc2 = { 0, 5, 10, 5, 0 };
     rAllocation =
-            createReservationAllocation(reservationID2, start, alloc2, true);
+        createReservationAllocation(reservationID2, start, alloc2, true);
     Assert.assertNull(plan.getReservationById(reservationID2));
     try {
       plan.addReservation(rAllocation, false);
@@ -367,16 +493,18 @@ public class TestInMemoryPlan {
 
     for (int i = 0; i < alloc2.length; i++) {
       Assert.assertEquals(
-          Resource.newInstance(1024 * (alloc1[i] + alloc2[i] + i), alloc1[i]
-              + alloc2[i] + i), plan.getTotalCommittedResources(start + i));
+          Resource.newInstance(1024 * (alloc1[i] + alloc2[i] + i),
+              alloc1[i] + alloc2[i] + i),
+          plan.getTotalCommittedResources(start + i));
       Assert.assertEquals(
-          Resource.newInstance(1024 * (alloc1[i] + alloc2[i] + i), alloc1[i]
-              + alloc2[i] + i), userCons.getCapacityAtTime(start + i));
+          Resource.newInstance(1024 * (alloc1[i] + alloc2[i] + i),
+              alloc1[i] + alloc2[i] + i),
+          userCons.getCapacityAtTime(start + i));
     }
 
     // Now archive completed reservations
     when(clock.getTime()).thenReturn(106L);
-    when(policy.getValidWindow()).thenReturn(1L);
+    when(sharingPolicy.getValidWindow()).thenReturn(1L);
     try {
       // will only remove 2nd reservation as only that has fallen out of the
       // archival window
@@ -386,7 +514,7 @@ public class TestInMemoryPlan {
     }
     Assert.assertNotNull(plan.getReservationById(reservationID1));
     Assert.assertNull(plan.getReservationById(reservationID2));
-    checkAllocation(plan, alloc1, start);
+    checkAllocation(plan, alloc1, start, 0);
 
     when(clock.getTime()).thenReturn(107L);
     try {
@@ -411,15 +539,14 @@ public class TestInMemoryPlan {
 
   @Test
   public void testGetReservationsById() {
-    Plan plan =
-            new InMemoryPlan(queueMetrics, policy, agent, totalCapacity, 1L,
-                    resCalc, minAlloc, maxAlloc, planName, replanner, true, context);
+    Plan plan = new InMemoryPlan(queueMetrics, policy, agent, totalCapacity, 1L,
+        resCalc, minAlloc, maxAlloc, planName, replanner, true, context);
     ReservationId reservationID =
-            ReservationSystemTestUtil.getNewReservationId();
-    int[] alloc = {10, 10, 10, 10, 10, 10};
+        ReservationSystemTestUtil.getNewReservationId();
+    int[] alloc = { 10, 10, 10, 10, 10, 10 };
     int start = 100;
-    ReservationAllocation rAllocation = createReservationAllocation
-            (reservationID, start, alloc);
+    ReservationAllocation rAllocation =
+        createReservationAllocation(reservationID, start, alloc);
     Assert.assertNull(plan.getReservationById(reservationID));
     try {
       plan.addReservation(rAllocation, false);
@@ -429,31 +556,30 @@ public class TestInMemoryPlan {
 
     // Verify that get by reservation id works.
     Set<ReservationAllocation> rAllocations =
-            plan.getReservations(reservationID, null, "");
+        plan.getReservations(reservationID, null, "");
     Assert.assertTrue(rAllocations.size() == 1);
-    Assert.assertTrue(rAllocation.compareTo(
-            (ReservationAllocation) rAllocations.toArray()[0]) == 0);
+    Assert.assertTrue(rAllocation
+        .compareTo((ReservationAllocation) rAllocations.toArray()[0]) == 0);
 
     // Verify that get by reservation id works even when time range
     // and user is invalid.
     ReservationInterval interval = new ReservationInterval(0, 0);
     rAllocations = plan.getReservations(reservationID, interval, "invalid");
     Assert.assertTrue(rAllocations.size() == 1);
-    Assert.assertTrue(rAllocation.compareTo(
-            (ReservationAllocation) rAllocations.toArray()[0]) == 0);
+    Assert.assertTrue(rAllocation
+        .compareTo((ReservationAllocation) rAllocations.toArray()[0]) == 0);
   }
 
   @Test
   public void testGetReservationsByInvalidId() {
-    Plan plan =
-            new InMemoryPlan(queueMetrics, policy, agent, totalCapacity, 1L,
-                    resCalc, minAlloc, maxAlloc, planName, replanner, true, context);
+    Plan plan = new InMemoryPlan(queueMetrics, policy, agent, totalCapacity, 1L,
+        resCalc, minAlloc, maxAlloc, planName, replanner, true, context);
     ReservationId reservationID =
-            ReservationSystemTestUtil.getNewReservationId();
-    int[] alloc = {10, 10, 10, 10, 10, 10};
+        ReservationSystemTestUtil.getNewReservationId();
+    int[] alloc = { 10, 10, 10, 10, 10, 10 };
     int start = 100;
-    ReservationAllocation rAllocation = createReservationAllocation
-            (reservationID, start, alloc);
+    ReservationAllocation rAllocation =
+        createReservationAllocation(reservationID, start, alloc);
     Assert.assertNull(plan.getReservationById(reservationID));
     try {
       plan.addReservation(rAllocation, false);
@@ -463,23 +589,22 @@ public class TestInMemoryPlan {
 
     // If reservationId is null, then nothing is returned.
     ReservationId invalidReservationID =
-            ReservationSystemTestUtil.getNewReservationId();
+        ReservationSystemTestUtil.getNewReservationId();
     Set<ReservationAllocation> rAllocations =
-            plan.getReservations(invalidReservationID, null, "");
+        plan.getReservations(invalidReservationID, null, "");
     Assert.assertTrue(rAllocations.size() == 0);
   }
 
   @Test
   public void testGetReservationsByTimeInterval() {
-    Plan plan =
-            new InMemoryPlan(queueMetrics, policy, agent, totalCapacity, 1L,
-                    resCalc, minAlloc, maxAlloc, planName, replanner, true, context);
+    Plan plan = new InMemoryPlan(queueMetrics, policy, agent, totalCapacity, 1L,
+        resCalc, minAlloc, maxAlloc, planName, replanner, true, context);
     ReservationId reservationID =
-            ReservationSystemTestUtil.getNewReservationId();
-    int[] alloc = {10, 10, 10, 10, 10, 10};
+        ReservationSystemTestUtil.getNewReservationId();
+    int[] alloc = { 10, 10, 10, 10, 10, 10 };
     int start = 100;
-    ReservationAllocation rAllocation = createReservationAllocation
-            (reservationID, start, alloc);
+    ReservationAllocation rAllocation =
+        createReservationAllocation(reservationID, start, alloc);
     Assert.assertNull(plan.getReservationById(reservationID));
     try {
       plan.addReservation(rAllocation, false);
@@ -489,23 +614,24 @@ public class TestInMemoryPlan {
 
     // Verify that get by time interval works if the selection interval
     // completely overlaps with the allocation.
-    ReservationInterval interval = new ReservationInterval(rAllocation
-            .getStartTime(), rAllocation.getEndTime());
+    ReservationInterval interval = new ReservationInterval(
+        rAllocation.getStartTime(), rAllocation.getEndTime());
     Set<ReservationAllocation> rAllocations =
-            plan.getReservations(null, interval, "");
+        plan.getReservations(null, interval, "");
     Assert.assertTrue(rAllocations.size() == 1);
-    Assert.assertTrue(rAllocation.compareTo(
-            (ReservationAllocation) rAllocations.toArray()[0]) == 0);
+    Assert.assertTrue(rAllocation
+        .compareTo((ReservationAllocation) rAllocations.toArray()[0]) == 0);
 
     // Verify that get by time interval works if the selection interval
     // falls within the allocation
     long duration = rAllocation.getEndTime() - rAllocation.getStartTime();
-    interval = new ReservationInterval(rAllocation.getStartTime() + duration
-            * (long)0.3, rAllocation.getEndTime() - duration * (long)0.3);
+    interval = new ReservationInterval(
+        rAllocation.getStartTime() + duration * (long) 0.3,
+        rAllocation.getEndTime() - duration * (long) 0.3);
     rAllocations = plan.getReservations(null, interval, "");
     Assert.assertTrue(rAllocations.size() == 1);
-    Assert.assertTrue(rAllocation.compareTo(
-            (ReservationAllocation) rAllocations.toArray()[0]) == 0);
+    Assert.assertTrue(rAllocation
+        .compareTo((ReservationAllocation) rAllocations.toArray()[0]) == 0);
 
     // Verify that get by time interval selects 1 allocation if the end
     // time of the selection interval falls right at the start of the
@@ -513,13 +639,13 @@ public class TestInMemoryPlan {
     interval = new ReservationInterval(0, rAllocation.getStartTime());
     rAllocations = plan.getReservations(null, interval, "");
     Assert.assertTrue(rAllocations.size() == 1);
-    Assert.assertTrue(rAllocation.compareTo(
-            (ReservationAllocation) rAllocations.toArray()[0]) == 0);
+    Assert.assertTrue(rAllocation
+        .compareTo((ReservationAllocation) rAllocations.toArray()[0]) == 0);
 
     // Verify that get by time interval selects no reservations if the start
     // time of the selection interval falls right at the end of the allocation.
-    interval = new ReservationInterval(rAllocation
-            .getEndTime(), Long.MAX_VALUE);
+    interval =
+        new ReservationInterval(rAllocation.getEndTime(), Long.MAX_VALUE);
     rAllocations = plan.getReservations(null, interval, "");
     Assert.assertTrue(rAllocations.size() == 0);
 
@@ -532,15 +658,14 @@ public class TestInMemoryPlan {
 
   @Test
   public void testGetReservationsAtTime() {
-    Plan plan =
-            new InMemoryPlan(queueMetrics, policy, agent, totalCapacity, 1L,
-                    resCalc, minAlloc, maxAlloc, planName, replanner, true, context);
+    Plan plan = new InMemoryPlan(queueMetrics, policy, agent, totalCapacity, 1L,
+        resCalc, minAlloc, maxAlloc, planName, replanner, true, context);
     ReservationId reservationID =
-            ReservationSystemTestUtil.getNewReservationId();
-    int[] alloc = {10, 10, 10, 10, 10, 10};
+        ReservationSystemTestUtil.getNewReservationId();
+    int[] alloc = { 10, 10, 10, 10, 10, 10 };
     int start = 100;
-    ReservationAllocation rAllocation = createReservationAllocation
-            (reservationID, start, alloc);
+    ReservationAllocation rAllocation =
+        createReservationAllocation(reservationID, start, alloc);
     Assert.assertNull(plan.getReservationById(reservationID));
     try {
       plan.addReservation(rAllocation, false);
@@ -549,23 +674,22 @@ public class TestInMemoryPlan {
     }
 
     Set<ReservationAllocation> rAllocations =
-            plan.getReservationsAtTime(rAllocation.getStartTime());
+        plan.getReservationsAtTime(rAllocation.getStartTime());
     Assert.assertTrue(rAllocations.size() == 1);
-    Assert.assertTrue(rAllocation.compareTo(
-            (ReservationAllocation) rAllocations.toArray()[0]) == 0);
+    Assert.assertTrue(rAllocation
+        .compareTo((ReservationAllocation) rAllocations.toArray()[0]) == 0);
   }
 
   @Test
   public void testGetReservationsWithNoInput() {
-    Plan plan =
-            new InMemoryPlan(queueMetrics, policy, agent, totalCapacity, 1L,
-                    resCalc, minAlloc, maxAlloc, planName, replanner, true, context);
+    Plan plan = new InMemoryPlan(queueMetrics, policy, agent, totalCapacity, 1L,
+        resCalc, minAlloc, maxAlloc, planName, replanner, true, context);
     ReservationId reservationID =
-            ReservationSystemTestUtil.getNewReservationId();
-    int[] alloc = {10, 10, 10, 10, 10, 10};
+        ReservationSystemTestUtil.getNewReservationId();
+    int[] alloc = { 10, 10, 10, 10, 10, 10 };
     int start = 100;
-    ReservationAllocation rAllocation = createReservationAllocation
-            (reservationID, start, alloc);
+    ReservationAllocation rAllocation =
+        createReservationAllocation(reservationID, start, alloc);
     Assert.assertNull(plan.getReservationById(reservationID));
     try {
       plan.addReservation(rAllocation, false);
@@ -576,22 +700,21 @@ public class TestInMemoryPlan {
     // Verify that getReservations defaults to getting all reservations if no
     // reservationID, time interval, and user is provided,
     Set<ReservationAllocation> rAllocations =
-            plan.getReservations(null, null, "");
+        plan.getReservations(null, null, "");
     Assert.assertTrue(rAllocations.size() == 1);
-    Assert.assertTrue(rAllocation.compareTo(
-            (ReservationAllocation) rAllocations.toArray()[0]) == 0);
+    Assert.assertTrue(rAllocation
+        .compareTo((ReservationAllocation) rAllocations.toArray()[0]) == 0);
   }
 
   @Test
   public void testGetReservationsWithNoReservation() {
-    Plan plan =
-            new InMemoryPlan(queueMetrics, policy, agent, totalCapacity, 1L,
-                    resCalc, minAlloc, maxAlloc, planName, replanner, true, context);
+    Plan plan = new InMemoryPlan(queueMetrics, policy, agent, totalCapacity, 1L,
+        resCalc, minAlloc, maxAlloc, planName, replanner, true, context);
     // Verify that get reservation returns no entries if no queries are made.
 
     ReservationInterval interval = new ReservationInterval(0, Long.MAX_VALUE);
     Set<ReservationAllocation> rAllocations =
-            plan.getReservations(null, interval, "");
+        plan.getReservations(null, interval, "");
     Assert.assertTrue(rAllocations.size() == 0);
   }
 
@@ -600,7 +723,9 @@ public class TestInMemoryPlan {
     Assert.assertNotNull(plan.getReservationById(reservationID));
     Assert.assertEquals(rAllocation, plan.getReservationById(reservationID));
     Assert.assertTrue(((InMemoryPlan) plan).getAllReservations().size() == 1);
-    Assert.assertEquals(rAllocation.getEndTime(), plan.getLastEndTime());
+    if (rAllocation.getPeriodicity() <= 0) {
+      Assert.assertEquals(rAllocation.getEndTime(), plan.getLastEndTime());
+    }
     Assert.assertEquals(totalCapacity, plan.getTotalCapacity());
     Assert.assertEquals(minAlloc, plan.getMinimumAllocation());
     Assert.assertEquals(maxAlloc, plan.getMaximumAllocation());
@@ -610,7 +735,8 @@ public class TestInMemoryPlan {
   }
 
   private ReservationDefinition createSimpleReservationDefinition(long arrival,
-      long deadline, long duration, Collection<ReservationRequest> resources) {
+      long deadline, long duration, Collection<ReservationRequest> resources,
+      String recurrenceExpression) {
     // create a request with a single atomic ask
     ReservationDefinition rDef = new ReservationDefinitionPBImpl();
     ReservationRequests reqs = new ReservationRequestsPBImpl();
@@ -619,6 +745,7 @@ public class TestInMemoryPlan {
     rDef.setReservationRequests(reqs);
     rDef.setArrival(arrival);
     rDef.setDeadline(deadline);
+    rDef.setRecurrenceExpression(recurrenceExpression);
     return rDef;
   }
 
@@ -633,31 +760,43 @@ public class TestInMemoryPlan {
       } else {
         numContainers = alloc[i];
       }
-      ReservationRequest rr =
-          ReservationRequest.newInstance(Resource.newInstance(1024, 1),
-              (numContainers));
+      ReservationRequest rr = ReservationRequest
+          .newInstance(Resource.newInstance(1024, 1), (numContainers));
       req.put(new ReservationInterval(startTime + i, startTime + i + 1), rr);
     }
     return req;
   }
 
-  private ReservationAllocation createReservationAllocation(ReservationId
-            reservationID, int start, int[] alloc) {
-    return createReservationAllocation(reservationID, start, alloc, false);
+  private ReservationAllocation createReservationAllocation(
+      ReservationId reservationID, int start, int[] alloc) {
+    return createReservationAllocation(reservationID, start, alloc, false, "0");
+  }
+
+  private ReservationAllocation createReservationAllocation(
+      ReservationId reservationID, int start, int[] alloc, boolean isStep) {
+    return createReservationAllocation(reservationID, start, alloc, isStep,
+        "0");
+  }
+
+  private ReservationAllocation createReservationAllocation(
+      ReservationId reservationID, int start, int[] alloc,
+      String recurrenceExp) {
+    return createReservationAllocation(reservationID, start, alloc, false,
+        recurrenceExp);
   }
 
-  private ReservationAllocation createReservationAllocation(ReservationId
-            reservationID, int start, int[] alloc, boolean isStep) {
+  private ReservationAllocation createReservationAllocation(
+      ReservationId reservationID, int start, int[] alloc, boolean isStep,
+      String recurrenceExp) {
     Map<ReservationInterval, ReservationRequest> allocations =
-            generateAllocation(start, alloc, isStep);
+        generateAllocation(start, alloc, isStep);
     ReservationDefinition rDef =
-            createSimpleReservationDefinition(start, start + alloc.length,
-                    alloc.length, allocations.values());
+        createSimpleReservationDefinition(start, start + alloc.length,
+            alloc.length, allocations.values(), recurrenceExp);
     Map<ReservationInterval, Resource> allocs =
-            ReservationSystemUtil.toResources(allocations);
+        ReservationSystemUtil.toResources(allocations);
     return new InMemoryReservationAllocation(reservationID, rDef, user,
-            planName,
-                    start, start + alloc.length, allocs, resCalc, minAlloc);
+        planName, start, start + alloc.length, allocs, resCalc, minAlloc);
   }
 
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b6e7d136/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestPeriodicRLESparseResourceAllocation.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/reservation/TestPeriodicRLESparseResourceAllocation.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestPeriodicRLESparseResourceAllocation.java
index 554eb58..457e2ee 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestPeriodicRLESparseResourceAllocation.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestPeriodicRLESparseResourceAllocation.java
@@ -19,26 +19,27 @@
 package org.apache.hadoop.yarn.server.resourcemanager.reservation;
 
 import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.server.resourcemanager.reservation.exceptions.PlanningException;
 import org.junit.Assert;
 import org.junit.Test;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 /**
- * Testing the class PeriodicRLESparseResourceAllocation.
+ * Testing the class {@link PeriodicRLESparseResourceAllocation}.
  */
+@SuppressWarnings("checkstyle:nowhitespaceafter")
 public class TestPeriodicRLESparseResourceAllocation {
 
-  private static final Logger LOG = LoggerFactory
-      .getLogger(TestPeriodicRLESparseResourceAllocation.class);
+  private static final Logger LOG =
+      LoggerFactory.getLogger(TestPeriodicRLESparseResourceAllocation.class);
 
   @Test
   public void testPeriodicCapacity() {
-    int[] alloc = {10, 7, 5, 2, 0};
-    long[] timeSteps = {0L, 5L, 10L, 15L, 19L};
-    RLESparseResourceAllocation rleSparseVector =
-        ReservationSystemTestUtil.generateRLESparseResourceAllocation(
-            alloc, timeSteps);
+    int[] alloc = { 10, 7, 5, 2, 0 };
+    long[] timeSteps = { 0L, 5L, 10L, 15L, 19L };
+    RLESparseResourceAllocation rleSparseVector = ReservationSystemTestUtil
+        .generateRLESparseResourceAllocation(alloc, timeSteps);
     PeriodicRLESparseResourceAllocation periodicVector =
         new PeriodicRLESparseResourceAllocation(rleSparseVector, 20L);
     LOG.info(periodicVector.toString());
@@ -54,43 +55,74 @@ public class TestPeriodicRLESparseResourceAllocation {
 
   @Test
   public void testMaxPeriodicCapacity() {
-    int[] alloc = {2, 5, 7, 10, 3, 4, 6, 8};
-    long[] timeSteps = {0L, 1L, 2L, 3L, 4L, 5L, 6L, 7L};
-    RLESparseResourceAllocation rleSparseVector =
-        ReservationSystemTestUtil.generateRLESparseResourceAllocation(
-            alloc, timeSteps);
+    int[] alloc = { 2, 5, 7, 10, 3, 4, 6, 8 };
+    long[] timeSteps = { 0L, 1L, 2L, 3L, 4L, 5L, 6L, 7L };
+    RLESparseResourceAllocation rleSparseVector = ReservationSystemTestUtil
+        .generateRLESparseResourceAllocation(alloc, timeSteps);
     PeriodicRLESparseResourceAllocation periodicVector =
         new PeriodicRLESparseResourceAllocation(rleSparseVector, 8L);
     LOG.info(periodicVector.toString());
-    Assert.assertEquals(
-        periodicVector.getMaximumPeriodicCapacity(0, 1),
+    Assert.assertEquals(periodicVector.getMaximumPeriodicCapacity(0, 1),
         Resource.newInstance(10, 10));
-    Assert.assertEquals(
-        periodicVector.getMaximumPeriodicCapacity(8, 2),
+    Assert.assertEquals(periodicVector.getMaximumPeriodicCapacity(8, 2),
         Resource.newInstance(7, 7));
-    Assert.assertEquals(
-        periodicVector.getMaximumPeriodicCapacity(16, 3),
+    Assert.assertEquals(periodicVector.getMaximumPeriodicCapacity(16, 3),
         Resource.newInstance(10, 10));
-    Assert.assertEquals(
-        periodicVector.getMaximumPeriodicCapacity(17, 4),
+    Assert.assertEquals(periodicVector.getMaximumPeriodicCapacity(17, 4),
         Resource.newInstance(5, 5));
-    Assert.assertEquals(
-        periodicVector.getMaximumPeriodicCapacity(32, 5),
+    Assert.assertEquals(periodicVector.getMaximumPeriodicCapacity(32, 5),
         Resource.newInstance(4, 4));
   }
 
   @Test
+  public void testMixPeriodicAndNonPeriodic() throws PlanningException {
+    int[] alloc = { 2, 5, 0 };
+    long[] timeSteps = { 1L, 2L, 3L };
+    RLESparseResourceAllocation tempPeriodic = ReservationSystemTestUtil
+        .generateRLESparseResourceAllocation(alloc, timeSteps);
+    PeriodicRLESparseResourceAllocation periodic =
+        new PeriodicRLESparseResourceAllocation(tempPeriodic, 10L);
+
+    int[] alloc2 = { 10, 10, 0 };
+    long[] timeSteps2 = { 12L, 13L, 14L };
+    RLESparseResourceAllocation nonPeriodic = ReservationSystemTestUtil
+        .generateRLESparseResourceAllocation(alloc2, timeSteps2);
+
+    RLESparseResourceAllocation merged =
+        RLESparseResourceAllocation.merge(nonPeriodic.getResourceCalculator(),
+            Resource.newInstance(100 * 1024, 100), periodic, nonPeriodic,
+            RLESparseResourceAllocation.RLEOperator.add, 2, 25);
+
+    Assert.assertEquals(Resource.newInstance(5, 5),
+        merged.getCapacityAtTime(2L));
+    Assert.assertEquals(Resource.newInstance(0, 0),
+        merged.getCapacityAtTime(3L));
+    Assert.assertEquals(Resource.newInstance(2, 2),
+        merged.getCapacityAtTime(11L));
+    Assert.assertEquals(Resource.newInstance(15, 15),
+        merged.getCapacityAtTime(12L));
+    Assert.assertEquals(Resource.newInstance(10, 10),
+        merged.getCapacityAtTime(13L));
+    Assert.assertEquals(Resource.newInstance(0, 0),
+        merged.getCapacityAtTime(14L));
+    Assert.assertEquals(Resource.newInstance(2, 2),
+        merged.getCapacityAtTime(21L));
+    Assert.assertEquals(Resource.newInstance(5, 5),
+        merged.getCapacityAtTime(22L));
+    Assert.assertEquals(Resource.newInstance(0, 0),
+        merged.getCapacityAtTime(23L));
+  }
+
+  @Test
   public void testSetCapacityInInterval() {
-    int[] alloc = {2, 5, 0};
-    long[] timeSteps = {1L, 2L, 3L};
-    RLESparseResourceAllocation rleSparseVector =
-        ReservationSystemTestUtil.generateRLESparseResourceAllocation(
-            alloc, timeSteps);
+    int[] alloc = { 2, 5, 0 };
+    long[] timeSteps = { 1L, 2L, 3L };
+    RLESparseResourceAllocation rleSparseVector = ReservationSystemTestUtil
+        .generateRLESparseResourceAllocation(alloc, timeSteps);
     PeriodicRLESparseResourceAllocation periodicVector =
         new PeriodicRLESparseResourceAllocation(rleSparseVector, 10L);
     ReservationInterval interval = new ReservationInterval(5L, 10L);
-    periodicVector.addInterval(
-        interval, Resource.newInstance(8, 8));
+    periodicVector.addInterval(interval, Resource.newInstance(8, 8));
     Assert.assertEquals(Resource.newInstance(8, 8),
         periodicVector.getCapacityAtTime(5L));
     Assert.assertEquals(Resource.newInstance(8, 8),
@@ -99,21 +131,20 @@ public class TestPeriodicRLESparseResourceAllocation {
         periodicVector.getCapacityAtTime(10L));
     Assert.assertEquals(Resource.newInstance(0, 0),
         periodicVector.getCapacityAtTime(0L));
-    Assert.assertFalse(periodicVector.addInterval(
-        new ReservationInterval(7L, 12L), Resource.newInstance(8, 8)));
+    // Assert.assertFalse(periodicVector.addInterval(
+    // new ReservationInterval(7L, 12L), Resource.newInstance(8, 8)));
   }
 
   public void testRemoveInterval() {
-    int[] alloc = {2, 5, 3, 4, 0};
-    long[] timeSteps = {1L, 3L, 5L, 7L, 9L};
-    RLESparseResourceAllocation rleSparseVector =
-        ReservationSystemTestUtil.generateRLESparseResourceAllocation(
-            alloc, timeSteps);
+    int[] alloc = { 2, 5, 3, 4, 0 };
+    long[] timeSteps = { 1L, 3L, 5L, 7L, 9L };
+    RLESparseResourceAllocation rleSparseVector = ReservationSystemTestUtil
+        .generateRLESparseResourceAllocation(alloc, timeSteps);
     PeriodicRLESparseResourceAllocation periodicVector =
         new PeriodicRLESparseResourceAllocation(rleSparseVector, 10L);
     ReservationInterval interval = new ReservationInterval(3L, 7L);
-    Assert.assertTrue(periodicVector.removeInterval(
-        interval, Resource.newInstance(3, 3)));
+    Assert.assertTrue(
+        periodicVector.removeInterval(interval, Resource.newInstance(3, 3)));
     Assert.assertEquals(Resource.newInstance(2, 2),
         periodicVector.getCapacityAtTime(1L));
     Assert.assertEquals(Resource.newInstance(2, 2),

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b6e7d136/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestRLESparseResourceAllocation.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/reservation/TestRLESparseResourceAllocation.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestRLESparseResourceAllocation.java
index bfe46e1..0027ceb 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestRLESparseResourceAllocation.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestRLESparseResourceAllocation.java
@@ -40,10 +40,14 @@ import org.junit.Test;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+/**
+ * Testing the class {@link RLESparseResourceAllocation}.
+ */
+@SuppressWarnings("checkstyle:nowhitespaceafter")
 public class TestRLESparseResourceAllocation {
 
-  private static final Logger LOG = LoggerFactory
-      .getLogger(TestRLESparseResourceAllocation.class);
+  private static final Logger LOG =
+      LoggerFactory.getLogger(TestRLESparseResourceAllocation.class);
 
   @Test
   public void testMergeAdd() throws PlanningException {
@@ -196,7 +200,8 @@ public class TestRLESparseResourceAllocation {
       // Expected!
     }
 
-    // Testing that the subtractTestNonNegative detects problems even if only one
+    // Testing that the subtractTestNonNegative detects problems even if only
+    // one
     // of the resource dimensions is "<0"
     a.put(10L, Resource.newInstance(10, 5));
     b.put(11L, Resource.newInstance(5, 6));
@@ -286,9 +291,8 @@ public class TestRLESparseResourceAllocation {
   public void testRangeOverlapping() {
     ResourceCalculator resCalc = new DefaultResourceCalculator();
 
-    RLESparseResourceAllocation r =
-        new RLESparseResourceAllocation(resCalc);
-    int[] alloc = {10, 10, 10, 10, 10, 10};
+    RLESparseResourceAllocation r = new RLESparseResourceAllocation(resCalc);
+    int[] alloc = { 10, 10, 10, 10, 10, 10 };
     int start = 100;
     Set<Entry<ReservationInterval, Resource>> inputs =
         generateAllocation(start, alloc, false).entrySet();
@@ -299,9 +303,9 @@ public class TestRLESparseResourceAllocation {
     long d = r.getLatestNonNullTime();
 
     // tries to trigger "out-of-range" bug
-    r =  r.getRangeOverlapping(s, d);
-    r = r.getRangeOverlapping(s-1, d-1);
-    r = r.getRangeOverlapping(s+1, d+1);
+    r = r.getRangeOverlapping(s, d);
+    r = r.getRangeOverlapping(s - 1, d - 1);
+    r = r.getRangeOverlapping(s + 1, d + 1);
   }
 
   @Test
@@ -370,25 +374,29 @@ public class TestRLESparseResourceAllocation {
     // Current bug prevents this to pass. The RLESparseResourceAllocation
     // does not handle removal of "partial"
     // allocations correctly.
-    Assert.assertEquals(102400, rleSparseVector.getCapacityAtTime(10)
-        .getMemorySize());
-    Assert.assertEquals(0, rleSparseVector.getCapacityAtTime(13).getMemorySize());
-    Assert.assertEquals(0, rleSparseVector.getCapacityAtTime(19).getMemorySize());
-    Assert.assertEquals(102400, rleSparseVector.getCapacityAtTime(21)
-        .getMemorySize());
-    Assert.assertEquals(2 * 102400, rleSparseVector.getCapacityAtTime(26)
-        .getMemorySize());
+    Assert.assertEquals(102400,
+        rleSparseVector.getCapacityAtTime(10).getMemorySize());
+    Assert.assertEquals(0,
+        rleSparseVector.getCapacityAtTime(13).getMemorySize());
+    Assert.assertEquals(0,
+        rleSparseVector.getCapacityAtTime(19).getMemorySize());
+    Assert.assertEquals(102400,
+        rleSparseVector.getCapacityAtTime(21).getMemorySize());
+    Assert.assertEquals(2 * 102400,
+        rleSparseVector.getCapacityAtTime(26).getMemorySize());
 
     ReservationInterval riRemove2 = new ReservationInterval(9, 13);
     rleSparseVector.removeInterval(riRemove2, rr);
     LOG.info(rleSparseVector.toString());
 
-    Assert.assertEquals(0, rleSparseVector.getCapacityAtTime(11).getMemorySize());
-    Assert.assertEquals(-102400, rleSparseVector.getCapacityAtTime(9)
-        .getMemorySize());
-    Assert.assertEquals(0, rleSparseVector.getCapacityAtTime(13).getMemorySize());
-    Assert.assertEquals(102400, rleSparseVector.getCapacityAtTime(20)
-        .getMemorySize());
+    Assert.assertEquals(0,
+        rleSparseVector.getCapacityAtTime(11).getMemorySize());
+    Assert.assertEquals(-102400,
+        rleSparseVector.getCapacityAtTime(9).getMemorySize());
+    Assert.assertEquals(0,
+        rleSparseVector.getCapacityAtTime(13).getMemorySize());
+    Assert.assertEquals(102400,
+        rleSparseVector.getCapacityAtTime(20).getMemorySize());
 
   }
 
@@ -500,7 +508,8 @@ public class TestRLESparseResourceAllocation {
     }
     mapAllocations = rleSparseVector.toIntervalMap();
     Assert.assertTrue(mapAllocations.size() == 5);
-    for (Entry<ReservationInterval, Resource> entry : mapAllocations.entrySet()) {
+    for (Entry<ReservationInterval, Resource> entry : mapAllocations
+        .entrySet()) {
       ReservationInterval interval = entry.getKey();
       Resource resource = entry.getValue();
       if (interval.getStartTime() == 101L) {
@@ -526,59 +535,46 @@ public class TestRLESparseResourceAllocation {
 
   @Test
   public void testMaxPeriodicCapacity() {
-    long[] timeSteps = {0L, 1L, 2L, 3L, 4L, 5L, 6L, 7L};
-    int[] alloc = {2, 5, 7, 10, 3, 4, 6, 8};
-    RLESparseResourceAllocation rleSparseVector =
-        ReservationSystemTestUtil.generateRLESparseResourceAllocation(
-            alloc, timeSteps);
+    long[] timeSteps = { 0L, 1L, 2L, 3L, 4L, 5L, 6L, 7L };
+    int[] alloc = { 2, 5, 7, 10, 3, 4, 6, 8 };
+    RLESparseResourceAllocation rleSparseVector = ReservationSystemTestUtil
+        .generateRLESparseResourceAllocation(alloc, timeSteps);
     LOG.info(rleSparseVector.toString());
-    Assert.assertEquals(
-        rleSparseVector.getMaximumPeriodicCapacity(0, 1),
+    Assert.assertEquals(rleSparseVector.getMaximumPeriodicCapacity(0, 1),
         Resource.newInstance(10, 10));
-    Assert.assertEquals(
-        rleSparseVector.getMaximumPeriodicCapacity(0, 2),
+    Assert.assertEquals(rleSparseVector.getMaximumPeriodicCapacity(0, 2),
         Resource.newInstance(7, 7));
-    Assert.assertEquals(
-        rleSparseVector.getMaximumPeriodicCapacity(0, 3),
+    Assert.assertEquals(rleSparseVector.getMaximumPeriodicCapacity(0, 3),
         Resource.newInstance(10, 10));
-    Assert.assertEquals(
-        rleSparseVector.getMaximumPeriodicCapacity(0, 4),
+    Assert.assertEquals(rleSparseVector.getMaximumPeriodicCapacity(0, 4),
         Resource.newInstance(3, 3));
-    Assert.assertEquals(
-        rleSparseVector.getMaximumPeriodicCapacity(0, 5),
+    Assert.assertEquals(rleSparseVector.getMaximumPeriodicCapacity(0, 5),
         Resource.newInstance(4, 4));
-    Assert.assertEquals(
-        rleSparseVector.getMaximumPeriodicCapacity(0, 5),
+    Assert.assertEquals(rleSparseVector.getMaximumPeriodicCapacity(0, 5),
         Resource.newInstance(4, 4));
-    Assert.assertEquals(
-        rleSparseVector.getMaximumPeriodicCapacity(7, 5),
+    Assert.assertEquals(rleSparseVector.getMaximumPeriodicCapacity(7, 5),
         Resource.newInstance(8, 8));
-    Assert.assertEquals(
-        rleSparseVector.getMaximumPeriodicCapacity(10, 3),
+    Assert.assertEquals(rleSparseVector.getMaximumPeriodicCapacity(10, 3),
         Resource.newInstance(0, 0));
   }
 
   @Test
   public void testGetMinimumCapacityInInterval() {
-    long[] timeSteps = {0L, 1L, 2L, 3L, 4L, 5L, 6L, 7L};
-    int[] alloc = {2, 5, 7, 10, 3, 4, 0, 8};
-    RLESparseResourceAllocation rleSparseVector =
-        ReservationSystemTestUtil.generateRLESparseResourceAllocation(
-            alloc, timeSteps);
+    long[] timeSteps = { 0L, 1L, 2L, 3L, 4L, 5L, 6L, 7L };
+    int[] alloc = { 2, 5, 7, 10, 3, 4, 0, 8 };
+    RLESparseResourceAllocation rleSparseVector = ReservationSystemTestUtil
+        .generateRLESparseResourceAllocation(alloc, timeSteps);
     LOG.info(rleSparseVector.toString());
-    Assert.assertEquals(
-        rleSparseVector.getMinimumCapacityInInterval(
-            new ReservationInterval(1L, 3L)), Resource.newInstance(5, 5));
-    Assert.assertEquals(
-        rleSparseVector.getMinimumCapacityInInterval(
-            new ReservationInterval(2L, 5L)), Resource.newInstance(3, 3));
-    Assert.assertEquals(
-        rleSparseVector.getMinimumCapacityInInterval(
-            new ReservationInterval(1L, 7L)), Resource.newInstance(0, 0));
+    Assert.assertEquals(rleSparseVector.getMinimumCapacityInInterval(
+        new ReservationInterval(1L, 3L)), Resource.newInstance(5, 5));
+    Assert.assertEquals(rleSparseVector.getMinimumCapacityInInterval(
+        new ReservationInterval(2L, 5L)), Resource.newInstance(3, 3));
+    Assert.assertEquals(rleSparseVector.getMinimumCapacityInInterval(
+        new ReservationInterval(1L, 7L)), Resource.newInstance(0, 0));
   }
 
-  private void setupArrays(
-      TreeMap<Long, Resource> a, TreeMap<Long, Resource> b) {
+  private void setupArrays(TreeMap<Long, Resource> a,
+      TreeMap<Long, Resource> b) {
     a.put(10L, Resource.newInstance(5, 5));
     a.put(20L, Resource.newInstance(10, 10));
     a.put(30L, Resource.newInstance(15, 15));
@@ -620,8 +616,8 @@ public class TestRLESparseResourceAllocation {
         numContainers = alloc[i];
       }
       req.put(new ReservationInterval(startTime + i, startTime + i + 1),
-          ReservationSystemUtil.toResource(ReservationRequest.newInstance(
-              Resource.newInstance(1024, 1), (numContainers))));
+          ReservationSystemUtil.toResource(ReservationRequest
+              .newInstance(Resource.newInstance(1024, 1), (numContainers))));
     }
     return req;
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b6e7d136/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/planning/TestSimpleCapacityReplanner.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/reservation/planning/TestSimpleCapacityReplanner.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/planning/TestSimpleCapacityReplanner.java
index c4f94c2..ddd290d 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/planning/TestSimpleCapacityReplanner.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/planning/TestSimpleCapacityReplanner.java
@@ -31,6 +31,7 @@ import org.apache.hadoop.yarn.api.records.ReservationDefinition;
 import org.apache.hadoop.yarn.api.records.ReservationId;
 import org.apache.hadoop.yarn.api.records.ReservationRequest;
 import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
 import org.apache.hadoop.yarn.server.resourcemanager.reservation.InMemoryPlan;
 import org.apache.hadoop.yarn.server.resourcemanager.reservation.InMemoryReservationAllocation;
@@ -78,9 +79,10 @@ public class TestSimpleCapacityReplanner {
     enf.init("blah", conf);
 
     // Initialize the plan with more resources
-    InMemoryPlan plan =
-        new InMemoryPlan(queueMetrics, policy, agent, clusterCapacity, step,
-            res, minAlloc, maxAlloc, "dedicated", enf, true, context, clock);
+    InMemoryPlan plan = new InMemoryPlan(queueMetrics, policy, agent,
+        clusterCapacity, step, res, minAlloc, maxAlloc, "dedicated", enf, true,
+        YarnConfiguration.DEFAULT_RM_RESERVATION_SYSTEM_MAX_PERIODICITY,
+        context, clock);
 
     // add reservation filling the plan (separating them 1ms, so we are sure
     // s2 follows s1 on acceptance


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[36/37] hadoop git commit: HDFS-12350. Support meta tags in configs. Contributed by Ajay Kumar.

Posted by ae...@apache.org.
HDFS-12350. Support meta tags in configs. Contributed by Ajay Kumar.


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

Branch: refs/heads/HDFS-7240
Commit: a4cd101934ae5a5cad9663de872fb4ecee0d7560
Parents: 83449ab
Author: Anu Engineer <ae...@apache.org>
Authored: Thu Sep 7 12:40:09 2017 -0700
Committer: Anu Engineer <ae...@apache.org>
Committed: Thu Sep 7 12:40:09 2017 -0700

----------------------------------------------------------------------
 .../org/apache/hadoop/conf/Configuration.java   | 116 ++++++++++++++++++-
 .../org/apache/hadoop/conf/CorePropertyTag.java |  37 ++++++
 .../org/apache/hadoop/conf/HDFSPropertyTag.java |  41 +++++++
 .../org/apache/hadoop/conf/PropertyTag.java     |  30 +++++
 .../org/apache/hadoop/conf/YarnPropertyTag.java |  39 +++++++
 .../apache/hadoop/conf/TestConfiguration.java   | 112 +++++++++++++++++-
 6 files changed, 373 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/a4cd1019/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/conf/Configuration.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/conf/Configuration.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/conf/Configuration.java
index a1a40b9..a339dac 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/conf/Configuration.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/conf/Configuration.java
@@ -259,7 +259,18 @@ public class Configuration implements Iterable<Map.Entry<String,String>>,
    */
   private static final WeakHashMap<Configuration,Object> REGISTRY = 
     new WeakHashMap<Configuration,Object>();
-  
+
+  /**
+   * Map to register all classes holding property tag enums.
+   */
+  private static final Map<String, Class>
+      REGISTERED_TAG_CLASS = new HashMap<>();
+  /**
+   * Map to hold properties by there tag groupings.
+   */
+  private final Map<PropertyTag, Properties> propertyTagsMap =
+      new ConcurrentHashMap<>();
+
   /**
    * List of default Resources. Resources are loaded in the order of the list 
    * entries
@@ -738,6 +749,12 @@ public class Configuration implements Iterable<Map.Entry<String,String>>,
   public Configuration(boolean loadDefaults) {
     this.loadDefaults = loadDefaults;
     updatingResource = new ConcurrentHashMap<String, String[]>();
+
+    // Register all classes holding property tags with
+    REGISTERED_TAG_CLASS.put("core", CorePropertyTag.class);
+    REGISTERED_TAG_CLASS.put("hdfs", HDFSPropertyTag.class);
+    REGISTERED_TAG_CLASS.put("yarn", YarnPropertyTag.class);
+
     synchronized(Configuration.class) {
       REGISTRY.put(this, null);
     }
@@ -765,6 +782,8 @@ public class Configuration implements Iterable<Map.Entry<String,String>>,
      this.finalParameters = Collections.newSetFromMap(
          new ConcurrentHashMap<String, Boolean>());
      this.finalParameters.addAll(other.finalParameters);
+     this.REGISTERED_TAG_CLASS.putAll(other.REGISTERED_TAG_CLASS);
+     this.propertyTagsMap.putAll(other.propertyTagsMap);
    }
    
     synchronized(Configuration.class) {
@@ -2823,6 +2842,10 @@ public class Configuration implements Iterable<Map.Entry<String,String>>,
               } else if ("source".equals(propertyAttr)) {
                 confSource.add(StringInterner.weakIntern(
                     reader.getAttributeValue(i)));
+              } else if ("tag".equals(propertyAttr)) {
+                //Read tags and put them in propertyTagsMap
+                readTagFromConfig(reader.getAttributeValue(i), confName,
+                    confValue, confSource);
               }
             }
             break;
@@ -2830,6 +2853,7 @@ public class Configuration implements Iterable<Map.Entry<String,String>>,
           case "value":
           case "final":
           case "source":
+          case "tag":
             parseToken = true;
             token.setLength(0);
             break;
@@ -2911,6 +2935,13 @@ public class Configuration implements Iterable<Map.Entry<String,String>>,
           case "source":
             confSource.add(StringInterner.weakIntern(token.toString()));
             break;
+          case "tag":
+            if (token.length() > 0) {
+              //Read tags and put them in propertyTagsMap
+              readTagFromConfig(token.toString(), confName,
+                  confValue, confSource);
+            }
+            break;
           case "include":
             if (fallbackAllowed && !fallbackEntered) {
               throw new IOException("Fetch fail on include for '"
@@ -2962,6 +2993,48 @@ public class Configuration implements Iterable<Map.Entry<String,String>>,
     }
   }
 
+  private void readTagFromConfig(String attributeValue, String confName, String
+      confValue, List<String> confSource) {
+    for (String tagStr : attributeValue.split(",")) {
+      tagStr = tagStr.trim();
+      try {
+        if (confSource.size() > 0) {
+          for (String source : confSource) {
+            PropertyTag tag1 = this.getPropertyTag(tagStr,
+                source.split("-")[0]);
+            if (propertyTagsMap.containsKey(tag1)) {
+              propertyTagsMap.get(tag1)
+                  .setProperty(confName, confValue);
+            } else {
+              Properties props = new Properties();
+              props.setProperty(confName, confValue);
+              propertyTagsMap.put(tag1, props);
+            }
+          }
+        } else {
+          //If no source is set try to find tag in CorePropertyTag
+          if (propertyTagsMap
+              .containsKey(CorePropertyTag.valueOf(tagStr)
+              )) {
+            propertyTagsMap.get(CorePropertyTag.valueOf(tagStr))
+                .setProperty(confName, confValue);
+          } else {
+            Properties props = new Properties();
+            props.setProperty(confName, confValue);
+            propertyTagsMap.put(CorePropertyTag.valueOf(tagStr),
+                props);
+          }
+        }
+      } catch (IllegalArgumentException iae) {
+        //Log the invalid tag and continue to parse rest of the
+        // properties.
+        LOG.info("Invalid tag '" + tagStr + "' found for "
+            + "property:" + confName, iae);
+      }
+
+    }
+  }
+
   private void overlay(Properties to, Properties from) {
     for (Entry<Object, Object> entry: from.entrySet()) {
       to.put(entry.getKey(), entry.getValue());
@@ -3438,4 +3511,45 @@ public class Configuration implements Iterable<Map.Entry<String,String>>,
     }
     return false;
   }
+
+  /**
+   * Get all properties belonging to tag.
+   * @return Properties with matching properties
+   */
+  public Properties getAllPropertiesByTag(final PropertyTag tag) {
+    Properties props = new Properties();
+    if (propertyTagsMap.containsKey(tag)) {
+      props.putAll(propertyTagsMap.get(tag));
+    }
+    return props;
+  }
+
+  /**
+   * Get all properties belonging to list of input tags. Calls
+   * getAllPropertiesByTag internally.
+   *
+   * @return Properties with all matching properties
+   */
+  public Properties getAllPropertiesByTags(final List<PropertyTag> tagList) {
+    Properties prop = new Properties();
+    for (PropertyTag tag : tagList) {
+      prop.putAll(this.getAllPropertiesByTag(tag));
+    }
+    return prop;
+  }
+
+  /**
+   * Get Property tag Enum corresponding to given source.
+   *
+   * @param tagStr String representation of Enum
+   * @param group Group to which enum belongs.Ex hdfs,yarn
+   * @return Properties with all matching properties
+   */
+  private PropertyTag getPropertyTag(String tagStr, String group) {
+    PropertyTag tag = null;
+    if (REGISTERED_TAG_CLASS.containsKey(group)) {
+      tag = (PropertyTag) Enum.valueOf(REGISTERED_TAG_CLASS.get(group), tagStr);
+    }
+    return tag;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a4cd1019/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/conf/CorePropertyTag.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/conf/CorePropertyTag.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/conf/CorePropertyTag.java
new file mode 100644
index 0000000..54a75b8
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/conf/CorePropertyTag.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.conf;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/***************************************************************************
+ * Enum for tagging hadoop core properties according to there usage.
+ * CorePropertyTag implements the
+ * {@link org.apache.hadoop.conf.PropertyTag} interface,
+ ***************************************************************************/
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public enum CorePropertyTag implements PropertyTag {
+  CORE,
+  REQUIRED,
+  PERFORMANCE,
+  CLIENT,
+  SERVER,
+  SECURITY,
+  DEBUG
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a4cd1019/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/conf/HDFSPropertyTag.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/conf/HDFSPropertyTag.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/conf/HDFSPropertyTag.java
new file mode 100644
index 0000000..02dfb86
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/conf/HDFSPropertyTag.java
@@ -0,0 +1,41 @@
+/**
+ * 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.conf;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/***************************************************************************
+ * Enum for tagging hdfs properties according to there usage or application.
+ * HDFSPropertyTag implements the
+ * {@link org.apache.hadoop.conf.PropertyTag} interface,
+ ***************************************************************************/
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public enum HDFSPropertyTag implements PropertyTag {
+  HDFS,
+  NAMENODE,
+  DATANODE,
+  REQUIRED,
+  SECURITY,
+  KERBEROS,
+  PERFORMANCE,
+  CLIENT,
+  SERVER,
+  DEBUG,
+  DEPRICATED
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a4cd1019/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/conf/PropertyTag.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/conf/PropertyTag.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/conf/PropertyTag.java
new file mode 100644
index 0000000..df8d4f9
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/conf/PropertyTag.java
@@ -0,0 +1,30 @@
+/**
+ * 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.conf;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/***********************************************************
+ * PropertyTag is used for creating extendable property tag Enums.
+ * Property tags will group related properties together.
+ ***********************************************************/
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public interface PropertyTag {
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a4cd1019/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/conf/YarnPropertyTag.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/conf/YarnPropertyTag.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/conf/YarnPropertyTag.java
new file mode 100644
index 0000000..e7a9c79
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/conf/YarnPropertyTag.java
@@ -0,0 +1,39 @@
+/**
+ * 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.conf;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/***************************************************************************
+ * Enum for tagging yarn properties according to there usage or application.
+ * YarnPropertyTag implements the
+ * {@link org.apache.hadoop.conf.PropertyTag} interface,
+ ***************************************************************************/
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public enum YarnPropertyTag implements PropertyTag {
+  YARN,
+  RESOURCEMANAGER,
+  SECURITY,
+  KERBEROS,
+  PERFORMANCE,
+  CLIENT,
+  REQUIRED,
+  SERVER,
+  DEBUG
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a4cd1019/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/conf/TestConfiguration.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/conf/TestConfiguration.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/conf/TestConfiguration.java
index b41a807..4cd1666 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/conf/TestConfiguration.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/conf/TestConfiguration.java
@@ -38,6 +38,7 @@ import java.util.HashMap;
 import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
+import java.util.Properties;
 import java.util.Random;
 import java.util.Set;
 import java.util.regex.Pattern;
@@ -53,7 +54,6 @@ import static org.junit.Assert.assertArrayEquals;
 
 import org.apache.commons.lang.StringUtils;
 import org.apache.hadoop.conf.Configuration.IntegerRanges;
-import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.IOUtils;
@@ -135,6 +135,7 @@ public class TestConfiguration {
 
   private void endConfig() throws IOException{
     out.write("</configuration>\n");
+    out.flush();
     out.close();
   }
 
@@ -577,6 +578,34 @@ public class TestConfiguration {
     out.write("</property>\n");
   }
 
+  private void appendPropertyByTag(String name, String val, String tags,
+      String... sources) throws IOException {
+    appendPropertyByTag(name, val, false, tags, sources);
+  }
+
+  private void appendPropertyByTag(String name, String val, boolean isFinal,
+      String tag, String... sources) throws IOException {
+    out.write("<property>");
+    out.write("<name>");
+    out.write(name);
+    out.write("</name>");
+    out.write("<value>");
+    out.write(val);
+    out.write("</value>");
+    if (isFinal) {
+      out.write("<final>true</final>");
+    }
+    for (String s : sources) {
+      out.write("<source>");
+      out.write(s);
+      out.write("</source>");
+    }
+    out.write("<tag>");
+    out.write(tag);
+    out.write("</tag>");
+    out.write("</property>\n");
+  }
+
   void appendCompactFormatProperty(String name, String val) throws IOException {
     appendCompactFormatProperty(name, val, false);
   }
@@ -2215,4 +2244,85 @@ public class TestConfiguration {
       TestConfiguration.class.getName()
     });
   }
+
+  @Test
+  public void testGetAllPropertiesByTags() throws Exception {
+
+    out = new BufferedWriter(new FileWriter(CONFIG));
+    startConfig();
+    appendPropertyByTag("dfs.cblock.trace.io", "false", "DEBUG");
+    appendPropertyByTag("dfs.replication", "1", "PERFORMANCE,REQUIRED");
+    appendPropertyByTag("dfs.namenode.logging.level", "INFO", "CLIENT,DEBUG");
+    endConfig();
+
+    Path fileResource = new Path(CONFIG);
+    conf.addResource(fileResource);
+    conf.getProps();
+
+    List<PropertyTag> tagList = new ArrayList<>();
+    tagList.add(CorePropertyTag.REQUIRED);
+    tagList.add(CorePropertyTag.PERFORMANCE);
+    tagList.add(CorePropertyTag.DEBUG);
+    tagList.add(CorePropertyTag.CLIENT);
+
+    Properties properties = conf.getAllPropertiesByTags(tagList);
+    assertEq(3, properties.size());
+    assertEq(true, properties.containsKey("dfs.namenode.logging.level"));
+    assertEq(true, properties.containsKey("dfs.replication"));
+    assertEq(true, properties.containsKey("dfs.cblock.trace.io"));
+    assertEq(false, properties.containsKey("namenode.host"));
+  }
+
+  @Test
+  public void testGetAllPropertiesWithSourceByTags() throws Exception {
+
+    out = new BufferedWriter(new FileWriter(CONFIG));
+    startConfig();
+    appendPropertyByTag("dfs.cblock.trace.io", "false", "DEBUG",
+        "hdfs-default.xml", "core-site.xml");
+    appendPropertyByTag("dfs.replication", "1", "PERFORMANCE,HDFS",
+        "hdfs-default.xml");
+    appendPropertyByTag("yarn.resourcemanager.work-preserving-recovery"
+        + ".enabled", "INFO", "CLIENT,DEBUG", "yarn-default.xml", "yarn-site"
+        + ".xml");
+    endConfig();
+
+    Path fileResource = new Path(CONFIG);
+    conf.addResource(fileResource);
+    conf.getProps();
+
+    List<PropertyTag> tagList = new ArrayList<>();
+    tagList.add(CorePropertyTag.REQUIRED);
+
+    Properties properties;
+    properties = conf.getAllPropertiesByTags(tagList);
+    assertNotEquals(3, properties.size());
+
+    tagList.add(HDFSPropertyTag.DEBUG);
+    tagList.add(YarnPropertyTag.CLIENT);
+    tagList.add(HDFSPropertyTag.PERFORMANCE);
+    tagList.add(HDFSPropertyTag.HDFS);
+    properties = conf.getAllPropertiesByTags(tagList);
+    assertEq(3, properties.size());
+
+    assertEq(true, properties.containsKey("dfs.cblock.trace.io"));
+    assertEq(true, properties.containsKey("dfs.replication"));
+    assertEq(true, properties
+        .containsKey("yarn.resourcemanager.work-preserving-recovery.enabled"));
+    assertEq(false, properties.containsKey("namenode.host"));
+
+    tagList.clear();
+    tagList.add(HDFSPropertyTag.DEBUG);
+    properties = conf.getAllPropertiesByTags(tagList);
+    assertEq(true, properties.containsKey("dfs.cblock.trace.io"));
+    assertEq(false, properties.containsKey("yarn.resourcemanager"
+        + ".work-preserving-recovery"));
+
+    tagList.clear();
+    tagList.add(YarnPropertyTag.DEBUG);
+    properties = conf.getAllPropertiesByTags(tagList);
+    assertEq(false, properties.containsKey("dfs.cblock.trace.io"));
+    assertEq(true, properties.containsKey("yarn.resourcemanager"
+        + ".work-preserving-recovery.enabled"));
+  }
 }


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[08/37] hadoop git commit: HDFS-12388. A bad error message in DFSStripedOutputStream. Contributed by Huafeng Wang

Posted by ae...@apache.org.
HDFS-12388. A bad error message in DFSStripedOutputStream. Contributed by Huafeng Wang


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

Branch: refs/heads/HDFS-7240
Commit: 5dba54596a1587e0ba5f9f02f40483e597b0df64
Parents: ed162b7
Author: Kai Zheng <ka...@intel.com>
Authored: Tue Sep 5 17:46:07 2017 +0800
Committer: Kai Zheng <ka...@intel.com>
Committed: Tue Sep 5 17:46:07 2017 +0800

----------------------------------------------------------------------
 .../main/java/org/apache/hadoop/hdfs/DFSStripedOutputStream.java   | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/5dba5459/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSStripedOutputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSStripedOutputStream.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSStripedOutputStream.java
index 842f23b..d5206d1 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSStripedOutputStream.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSStripedOutputStream.java
@@ -390,7 +390,7 @@ public class DFSStripedOutputStream extends DFSOutputStream
     }
     if (failCount > (numAllBlocks - numDataBlocks)) {
       throw new IOException("Failed: the number of failed blocks = "
-          + failCount + " > the number of data blocks = "
+          + failCount + " > the number of parity blocks = "
           + (numAllBlocks - numDataBlocks));
     }
     return newFailed;


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[15/37] hadoop git commit: HADOOP-14103. Sort out hadoop-aws contract-test-options.xml. Contributed by John Zhuge.

Posted by ae...@apache.org.
HADOOP-14103. Sort out hadoop-aws contract-test-options.xml. Contributed by John Zhuge.


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

Branch: refs/heads/HDFS-7240
Commit: 50506e90a843aaf1f771a32b872fdd7704b619f5
Parents: d7f2704
Author: John Zhuge <jz...@apache.org>
Authored: Tue Sep 5 23:26:31 2017 -0700
Committer: John Zhuge <jz...@apache.org>
Committed: Tue Sep 5 23:26:57 2017 -0700

----------------------------------------------------------------------
 .../site/markdown/tools/hadoop-aws/testing.md   | 129 +++----------------
 1 file changed, 18 insertions(+), 111 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/50506e90/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/testing.md
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/testing.md b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/testing.md
index 3b9b5c4..19d322d 100644
--- a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/testing.md
+++ b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/testing.md
@@ -76,13 +76,10 @@ over long-haul connections. Please help us identify and fix these problems
 
 ## Setting up the tests
 
-To integration test the S3* filesystem clients, you need to provide two files
-which pass in authentication details to the test runner.
+To integration test the S3* filesystem clients, you need to provide
+`auth-keys.xml` which passes in authentication details to the test runner.
 
-1. `auth-keys.xml`
-1. `contract-test-options.xml`
-
-These are both Hadoop XML configuration files, which must be placed into
+It is a Hadoop XML configuration file, which must be placed into
 `hadoop-tools/hadoop-aws/src/test/resources`.
 
 ### File `core-site.xml`
@@ -106,6 +103,8 @@ each filesystem for its testing.
 
 1. `test.fs.s3n.name` : the URL of the bucket for S3n tests
 1. `test.fs.s3a.name` : the URL of the bucket for S3a tests
+1. `fs.contract.test.fs.s3n` : the URL of the bucket for S3n filesystem contract tests
+1. `fs.contract.test.fs.s3a` : the URL of the bucket for S3a filesystem contract tests
 
 *Note* that running s3a and s3n tests in parallel mode, against the same bucket
 is unreliable.  We recommend using separate buckets or testing one connector
@@ -128,8 +127,8 @@ Example:
   </property>
 
   <property>
-    <name>test.fs.s3a.name</name>
-    <value>s3a://test-aws-s3a/</value>
+    <name>fs.contract.test.fs.s3n</name>
+    <value>${test.fs.s3n.name}</value>
   </property>
 
   <property>
@@ -143,6 +142,16 @@ Example:
   </property>
 
   <property>
+    <name>test.fs.s3a.name</name>
+    <value>s3a://test-aws-s3a/</value>
+  </property>
+
+  <property>
+    <name>fs.contract.test.fs.s3a</name>
+    <value>${test.fs.s3a.name}</value>
+  </property>
+
+  <property>
     <name>fs.s3a.access.key</name>
     <description>AWS access key ID. Omit for IAM role-based authentication.</description>
     <value>DONOTCOMMITTHISKEYTOSCM</value>
@@ -163,107 +172,6 @@ Example:
 </configuration>
 ```
 
-### File `contract-test-options.xml`
-
-The file `hadoop-tools/hadoop-aws/src/test/resources/contract-test-options.xml`
-must be created and configured for the test filesystems.
-
-If a specific file `fs.contract.test.fs.*` test path is not defined for
-any of the filesystems, those tests will be skipped.
-
-The standard S3 authentication details must also be provided. This can be
-through copy-and-paste of the `auth-keys.xml` credentials, or it can be
-through direct XInclude inclusion.
-
-Here is an an example `contract-test-options.xml` which places all test options
-into the `auth-keys.xml` file, so offering a single place to keep credentials
-and define test endpoint bindings.
-
-```xml
-<configuration>
-  <include xmlns="http://www.w3.org/2001/XInclude"
-    href="auth-keys.xml"/>
-</configuration>
-```
-
-### s3n://
-
-
-In the file `src/test/resources/contract-test-options.xml`, the filesystem
-name must be defined in the property `fs.contract.test.fs.s3n`.
-The standard configuration options to define the S3N authentication details
-must also be provided.
-
-Example:
-
-```xml
-<property>
-  <name>fs.contract.test.fs.s3n</name>
-  <value>s3n://test-aws-s3n/</value>
-</property>
-```
-
-### s3a://
-
-
-In the file `src/test/resources/contract-test-options.xml`, the filesystem
-name must be defined in the property `fs.contract.test.fs.s3a`.
-The standard configuration options to define the S3N authentication details
-must also be provided.
-
-Example:
-
-```xml
-<property>
-  <name>fs.contract.test.fs.s3a</name>
-  <value>s3a://test-aws-s3a/</value>
-</property>
-```
-
-### Complete example of `contract-test-options.xml`
-
-
-```xml
-<?xml version="1.0"?>
-<?xml-stylesheet type="text/xsl" href="configuration.xsl"?>
-<!---
-  Licensed 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. See accompanying LICENSE file.
--->
-
-<configuration>
-
-  <include xmlns="http://www.w3.org/2001/XInclude"
-  href="auth-keys.xml"/>
-
-  <property>
-    <name>fs.contract.test.fs.s3a</name>
-    <value>s3a://test-aws-s3a/</value>
-  </property>
-
-  <property>
-    <name>fs.contract.test.fs.s3n</name>
-    <value>s3n://test-aws-s3n/</value>
-  </property>
-
-</configuration>
-```
-
-This example pulls in the `auth-keys.xml` file for the credentials.
-This provides one single place to keep the keys up to date —and means
-that the file `contract-test-options.xml` does not contain any
-secret credentials itself. As the auth keys XML file is kept out of the
-source code tree, it is not going to get accidentally committed.
-
 ### Configuring S3a Encryption
 
 For S3a encryption tests to run correctly, the
@@ -346,8 +254,7 @@ like `ITestS3A*` shown above, it may cause unpredictable test failures.
 ### Testing against different regions
 
 S3A can connect to different regions —the tests support this. Simply
-define the target region in `contract-test-options.xml` or any `auth-keys.xml`
-file referenced.
+define the target region in `auth-keys.xml`.
 
 ```xml
 <property>


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[30/37] hadoop git commit: HDFS-12376. Enable JournalNode Sync by default. Contributed by Hanisha Koneru.

Posted by ae...@apache.org.
HDFS-12376. Enable JournalNode Sync by default. Contributed by Hanisha Koneru.


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

Branch: refs/heads/HDFS-7240
Commit: 5ff74167ddd378a738898425c4462f3432f9c31d
Parents: 2adf8be
Author: Arpit Agarwal <ar...@apache.org>
Authored: Thu Sep 7 08:18:28 2017 -0700
Committer: Arpit Agarwal <ar...@apache.org>
Committed: Thu Sep 7 08:18:28 2017 -0700

----------------------------------------------------------------------
 .../src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java        | 2 +-
 .../hadoop-hdfs/src/main/resources/hdfs-default.xml                | 2 +-
 2 files changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/5ff74167/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
index 17cabad..bc7b716 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
@@ -924,7 +924,7 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
   public static final String  DFS_JOURNALNODE_KERBEROS_INTERNAL_SPNEGO_PRINCIPAL_KEY = "dfs.journalnode.kerberos.internal.spnego.principal";
   public static final String DFS_JOURNALNODE_ENABLE_SYNC_KEY =
       "dfs.journalnode.enable.sync";
-  public static final boolean DFS_JOURNALNODE_ENABLE_SYNC_DEFAULT = false;
+  public static final boolean DFS_JOURNALNODE_ENABLE_SYNC_DEFAULT = true;
   public static final String DFS_JOURNALNODE_SYNC_INTERVAL_KEY =
       "dfs.journalnode.sync.interval";
   public static final long DFS_JOURNALNODE_SYNC_INTERVAL_DEFAULT = 2*60*1000L;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5ff74167/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 bbe2eca..d58e54e 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
@@ -3933,7 +3933,7 @@
 
 <property>
   <name>dfs.journalnode.enable.sync</name>
-  <value>false</value>
+  <value>true</value>
   <description>
     If true, the journal nodes wil sync with each other. The journal nodes
     will periodically gossip with other journal nodes to compare edit log


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[05/37] hadoop git commit: HDFS-12383. Re-encryption updater should handle canceled tasks better.

Posted by ae...@apache.org.
HDFS-12383. Re-encryption updater should handle canceled tasks better.


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

Branch: refs/heads/HDFS-7240
Commit: 633c1ea4554cae6cd684f60848497817d4ed3d82
Parents: 275980b
Author: Xiao Chen <xi...@apache.org>
Authored: Sun Sep 3 19:50:21 2017 -0700
Committer: Xiao Chen <xi...@apache.org>
Committed: Sun Sep 3 19:50:41 2017 -0700

----------------------------------------------------------------------
 .../server/namenode/ReencryptionUpdater.java    |  15 ++-
 .../hdfs/server/namenode/TestReencryption.java  | 105 +++++++++++++++++--
 2 files changed, 108 insertions(+), 12 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/633c1ea4/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ReencryptionUpdater.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ReencryptionUpdater.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ReencryptionUpdater.java
index 690a0e9..d641ea1 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ReencryptionUpdater.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ReencryptionUpdater.java
@@ -39,6 +39,7 @@ import java.util.Iterator;
 import java.util.LinkedList;
 import java.util.List;
 import java.util.ListIterator;
+import java.util.concurrent.CancellationException;
 import java.util.concurrent.CompletionService;
 import java.util.concurrent.ExecutionException;
 import java.util.concurrent.Future;
@@ -72,6 +73,7 @@ public final class ReencryptionUpdater implements Runnable {
   private final StopWatch throttleTimerLocked = new StopWatch();
 
   private volatile long faultRetryInterval = 60000;
+  private volatile boolean isRunning = false;
 
   /**
    * Class to track re-encryption submissions of a single zone. It contains
@@ -201,6 +203,11 @@ public final class ReencryptionUpdater implements Runnable {
     pauseZoneId = zoneId;
   }
 
+  @VisibleForTesting
+  boolean isRunning() {
+    return isRunning;
+  }
+
   private final FSDirectory dir;
   private final CompletionService<ReencryptionTask> batchService;
   private final ReencryptionHandler handler;
@@ -242,6 +249,7 @@ public final class ReencryptionUpdater implements Runnable {
 
   @Override
   public void run() {
+    isRunning = true;
     throttleTimerAll.start();
     while (true) {
       try {
@@ -250,11 +258,13 @@ public final class ReencryptionUpdater implements Runnable {
       } catch (InterruptedException ie) {
         LOG.warn("Re-encryption updater thread interrupted. Exiting.");
         Thread.currentThread().interrupt();
+        isRunning = false;
         return;
-      } catch (IOException ioe) {
-        LOG.warn("Re-encryption updater thread exception.", ioe);
+      } catch (IOException | CancellationException e) {
+        LOG.warn("Re-encryption updater thread exception.", e);
       } catch (Throwable t) {
         LOG.error("Re-encryption updater thread exiting.", t);
+        isRunning = false;
         return;
       }
     }
@@ -405,6 +415,7 @@ public final class ReencryptionUpdater implements Runnable {
     if (completed.isCancelled()) {
       LOG.debug("Skipped canceled re-encryption task for zone {}, last: {}",
           task.zoneId, task.lastFile);
+      return;
     }
 
     boolean shouldRetry;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/633c1ea4/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestReencryption.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestReencryption.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestReencryption.java
index 7ba3f91..4b5be2e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestReencryption.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestReencryption.java
@@ -30,6 +30,7 @@ import java.util.Set;
 import java.util.concurrent.Future;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.TimeoutException;
+import java.util.concurrent.atomic.AtomicBoolean;
 
 import com.google.common.base.Supplier;
 
@@ -1086,12 +1087,9 @@ public class TestReencryption {
     getEzManager().resumeReencryptForTesting();
 
     Thread.sleep(3000);
-    EncryptionZoneManager ezm = getEzManager();
-    ReencryptionHandler handler = (ReencryptionHandler) Whitebox
-        .getInternalState(ezm, "reencryptionHandler");
     Map<Long, ZoneSubmissionTracker> tasks =
         (Map<Long, ZoneSubmissionTracker>) Whitebox
-            .getInternalState(handler, "submissions");
+            .getInternalState(getHandler(), "submissions");
     List<Future> futures = new LinkedList<>();
     for (ZoneSubmissionTracker zst : tasks.values()) {
       for (Future f : zst.getTasks()) {
@@ -1494,6 +1492,88 @@ public class TestReencryption {
   }
 
   @Test
+  public void testCancelFuture() throws Exception {
+    final AtomicBoolean callableRunning = new AtomicBoolean(false);
+    class MyInjector extends EncryptionFaultInjector {
+      private volatile int exceptionCount = 0;
+
+      MyInjector(int numFailures) {
+        exceptionCount = numFailures;
+      }
+
+      @Override
+      public void reencryptEncryptedKeys() throws IOException {
+        if (exceptionCount > 0) {
+          exceptionCount--;
+          try {
+            callableRunning.set(true);
+            Thread.sleep(Long.MAX_VALUE);
+          } catch (InterruptedException ie) {
+            LOG.info("Fault injector interrupted", ie);
+          }
+        }
+      }
+    }
+    final MyInjector injector = new MyInjector(1);
+    EncryptionFaultInjector.instance = injector;
+
+    /* Setup test dir:
+     * /zones/zone/[0-9]
+     * /dir/f
+     */
+    final int len = 8196;
+    final Path zoneParent = new Path("/zones");
+    final Path zone = new Path(zoneParent, "zone");
+    fsWrapper.mkdir(zone, FsPermission.getDirDefault(), true);
+    dfsAdmin.createEncryptionZone(zone, TEST_KEY, NO_TRASH);
+    for (int i = 0; i < 10; ++i) {
+      DFSTestUtil
+          .createFile(fs, new Path(zone, Integer.toString(i)), len, (short) 1,
+              0xFEED);
+    }
+    final Path subdir = new Path("/dir");
+    fsWrapper.mkdir(subdir, FsPermission.getDirDefault(), true);
+    DFSTestUtil.createFile(fs, new Path(subdir, "f"), len, (short) 1, 0xFEED);
+
+    // re-encrypt 10 files, so 2 callables. Hang 1, pause the updater so the
+    // callable is taken from the executor but not processed.
+    fsn.getProvider().rollNewVersion(TEST_KEY);
+    fsn.getProvider().flush();
+    getEzManager().pauseReencryptForTesting();
+    dfsAdmin.reencryptEncryptionZone(zone, ReencryptAction.START);
+    waitForQueuedZones(1);
+    getEzManager().resumeReencryptForTesting();
+
+    LOG.info("Waiting for re-encrypt callables to run");
+    GenericTestUtils.waitFor(new Supplier<Boolean>() {
+      @Override
+      public Boolean get() {
+        return callableRunning.get();
+      }
+    }, 100, 10000);
+
+    getEzManager().pauseReencryptUpdaterForTesting();
+    dfsAdmin.reencryptEncryptionZone(zone, ReencryptAction.CANCEL);
+
+    // now resume updater and verify status.
+    getEzManager().resumeReencryptUpdaterForTesting();
+    waitForZoneCompletes(zone.toString());
+
+    RemoteIterator<ZoneReencryptionStatus> it =
+        dfsAdmin.listReencryptionStatus();
+    assertTrue(it.hasNext());
+    final ZoneReencryptionStatus zs = it.next();
+    assertEquals(zone.toString(), zs.getZoneName());
+    assertEquals(ZoneReencryptionStatus.State.Completed, zs.getState());
+    assertTrue(zs.isCanceled());
+    assertTrue(zs.getCompletionTime() > 0);
+    assertTrue(zs.getCompletionTime() > zs.getSubmissionTime());
+    assertEquals(0, zs.getFilesReencrypted());
+
+    assertTrue(getUpdater().isRunning());
+  }
+
+  @Test
   public void testReencryptCancelForUpdater() throws Exception {
     /* Setup test dir:
      * /zones/zone/[0-9]
@@ -1822,12 +1902,7 @@ public class TestReencryption {
     fsn.getProvider().rollNewVersion(TEST_KEY);
     fsn.getProvider().flush();
 
-    final EncryptionZoneManager ezm = getEzManager();
-    final ReencryptionHandler handler = (ReencryptionHandler) Whitebox
-        .getInternalState(ezm, "reencryptionHandler");
-    final ReencryptionUpdater updater = (ReencryptionUpdater) Whitebox
-        .getInternalState(handler, "reencryptionUpdater");
-    Whitebox.setInternalState(updater, "faultRetryInterval", 50);
+    Whitebox.setInternalState(getUpdater(), "faultRetryInterval", 50);
     dfsAdmin.reencryptEncryptionZone(zone, ReencryptAction.START);
     waitForReencryptedZones(1);
     assertEquals(0, injector.exceptionCount);
@@ -1844,4 +1919,14 @@ public class TestReencryption {
     assertEquals(10, zs.getFilesReencrypted());
     assertEquals(0, zs.getNumReencryptionFailures());
   }
+
+  private ReencryptionHandler getHandler() {
+    return (ReencryptionHandler) Whitebox
+        .getInternalState(getEzManager(), "reencryptionHandler");
+  }
+
+  private ReencryptionUpdater getUpdater() {
+    return (ReencryptionUpdater) Whitebox
+        .getInternalState(getHandler(), "reencryptionUpdater");
+  }
 }
\ No newline at end of file


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[18/37] hadoop git commit: YARN-7148. TestLogsCLI fails in trunk and branch-2 and javadoc error. Contributed by Xuan Gong.

Posted by ae...@apache.org.
YARN-7148. TestLogsCLI fails in trunk and branch-2 and javadoc error. Contributed by Xuan Gong.


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

Branch: refs/heads/HDFS-7240
Commit: 63231a2a3008794c5ebcbc9d3855638fc5f28216
Parents: 1f3bc63
Author: Junping Du <ju...@apache.org>
Authored: Wed Sep 6 13:19:34 2017 -0700
Committer: Junping Du <ju...@apache.org>
Committed: Wed Sep 6 13:19:34 2017 -0700

----------------------------------------------------------------------
 .../hadoop/yarn/client/cli/TestLogsCLI.java     | 16 ++--
 .../logaggregation/AggregatedLogFormat.java     | 78 ++++++++++----------
 .../logaggregation/LogAggregationUtils.java     | 32 ++++----
 .../yarn/logaggregation/LogCLIHelpers.java      | 41 ++++++++--
 .../LogAggregationFileController.java           | 11 +--
 .../tfile/LogAggregationTFileController.java    |  1 -
 6 files changed, 103 insertions(+), 76 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/63231a2a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestLogsCLI.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestLogsCLI.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestLogsCLI.java
index 509a790..d58732e 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestLogsCLI.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestLogsCLI.java
@@ -285,7 +285,7 @@ public class TestLogsCLI {
   @Test (timeout = 15000)
   public void testFetchFinishedApplictionLogs() throws Exception {
     String remoteLogRootDir = "target/logs/";
-    Configuration configuration = new Configuration();
+    Configuration configuration = new YarnConfiguration();
     configuration.setBoolean(YarnConfiguration.LOG_AGGREGATION_ENABLED, true);
     configuration
       .set(YarnConfiguration.NM_REMOTE_APP_LOG_DIR, remoteLogRootDir);
@@ -646,7 +646,7 @@ public class TestLogsCLI {
         any(ContainerId.class));
 
     // create local logs
-    Configuration configuration = new Configuration();
+    Configuration configuration = new YarnConfiguration();
     FileSystem fs = FileSystem.get(configuration);
     String rootLogDir = "target/LocalLogs";
     Path rootLogDirPath = new Path(rootLogDir);
@@ -790,7 +790,7 @@ public class TestLogsCLI {
     UserGroupInformation testUgi = UserGroupInformation
         .createRemoteUser(testUser);
 
-    Configuration configuration = new Configuration();
+    Configuration configuration = new YarnConfiguration();
     configuration.setBoolean(YarnConfiguration.LOG_AGGREGATION_ENABLED, true);
     configuration
         .set(YarnConfiguration.NM_REMOTE_APP_LOG_DIR, remoteLogRootDir);
@@ -921,7 +921,7 @@ public class TestLogsCLI {
   public void testLogsCLIWithInvalidArgs() throws Exception {
     String localDir = "target/SaveLogs";
     Path localPath = new Path(localDir);
-    Configuration configuration = new Configuration();
+    Configuration configuration = new YarnConfiguration();
     FileSystem fs = FileSystem.get(configuration);
     ApplicationId appId = ApplicationId.newInstance(0, 1);
     YarnClient mockYarnClient =
@@ -992,7 +992,7 @@ public class TestLogsCLI {
     String localDir = "target/SaveLogs";
     Path localPath = new Path(localDir);
 
-    Configuration configuration = new Configuration();
+    Configuration configuration = new YarnConfiguration();
     configuration.setBoolean(YarnConfiguration.LOG_AGGREGATION_ENABLED, true);
     configuration
         .set(YarnConfiguration.NM_REMOTE_APP_LOG_DIR, remoteLogRootDir);
@@ -1083,7 +1083,7 @@ public class TestLogsCLI {
   @Test (timeout = 15000)
   public void testPrintContainerLogMetadata() throws Exception {
     String remoteLogRootDir = "target/logs/";
-    Configuration configuration = new Configuration();
+    Configuration configuration = new YarnConfiguration();
     configuration.setBoolean(YarnConfiguration.LOG_AGGREGATION_ENABLED, true);
     configuration
       .set(YarnConfiguration.NM_REMOTE_APP_LOG_DIR, remoteLogRootDir);
@@ -1188,7 +1188,7 @@ public class TestLogsCLI {
   @Test (timeout = 15000)
   public void testListNodeInfo() throws Exception {
     String remoteLogRootDir = "target/logs/";
-    Configuration configuration = new Configuration();
+    Configuration configuration = new YarnConfiguration();
     configuration.setBoolean(YarnConfiguration.LOG_AGGREGATION_ENABLED, true);
     configuration
       .set(YarnConfiguration.NM_REMOTE_APP_LOG_DIR, remoteLogRootDir);
@@ -1239,7 +1239,7 @@ public class TestLogsCLI {
   @Test (timeout = 15000)
   public void testFetchApplictionLogsHar() throws Exception {
     String remoteLogRootDir = "target/logs/";
-    Configuration configuration = new Configuration();
+    Configuration configuration = new YarnConfiguration();
     configuration.setBoolean(YarnConfiguration.LOG_AGGREGATION_ENABLED, true);
     configuration
         .set(YarnConfiguration.NM_REMOTE_APP_LOG_DIR, remoteLogRootDir);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/63231a2a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/AggregatedLogFormat.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/AggregatedLogFormat.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/AggregatedLogFormat.java
index 3c1dcdc..9bec147 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/AggregatedLogFormat.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/AggregatedLogFormat.java
@@ -583,9 +583,9 @@ public class AggregatedLogFormat {
 
     /**
      * Returns the owner of the application.
-     * 
+     *
      * @return the application owner.
-     * @throws IOException
+     * @throws IOException if we can not get the application owner.
      */
     public String getApplicationOwner() throws IOException {
       TFile.Reader.Scanner ownerScanner = null;
@@ -610,9 +610,9 @@ public class AggregatedLogFormat {
     /**
      * Returns ACLs for the application. An empty map is returned if no ACLs are
      * found.
-     * 
+     *
      * @return a map of the Application ACLs.
-     * @throws IOException
+     * @throws IOException if we can not get the application acls.
      */
     public Map<ApplicationAccessType, String> getApplicationAcls()
         throws IOException {
@@ -656,9 +656,10 @@ public class AggregatedLogFormat {
     /**
      * Read the next key and return the value-stream.
      * 
-     * @param key
-     * @return the valueStream if there are more keys or null otherwise.
-     * @throws IOException
+     * @param key the log key
+     * @return the valueStream if there are more keys or null otherwise
+     * @throws IOException if we can not get the dataInputStream
+     * for the next key
      */
     public DataInputStream next(LogKey key) throws IOException {
       if (!this.atBeginning) {
@@ -683,10 +684,10 @@ public class AggregatedLogFormat {
      * Get a ContainerLogsReader to read the logs for
      * the specified container.
      *
-     * @param containerId
+     * @param containerId the containerId
      * @return object to read the container's logs or null if the
      *         logs could not be found
-     * @throws IOException
+     * @throws IOException if we can not get the container log reader.
      */
     @Private
     public ContainerLogsReader getContainerLogsReader(
@@ -752,10 +753,10 @@ public class AggregatedLogFormat {
 
     /**
      * Writes all logs for a single container to the provided writer.
-     * @param valueStream
-     * @param writer
-     * @param logUploadedTime
-     * @throws IOException
+     * @param valueStream the valueStream
+     * @param writer the log writer
+     * @param logUploadedTime the time stamp
+     * @throws IOException if we can not read the container logs.
      */
     public static void readAcontainerLogs(DataInputStream valueStream,
         Writer writer, long logUploadedTime) throws IOException {
@@ -780,9 +781,9 @@ public class AggregatedLogFormat {
 
     /**
      * Writes all logs for a single container to the provided writer.
-     * @param valueStream
-     * @param writer
-     * @throws IOException
+     * @param valueStream the value stream
+     * @param writer the log writer
+     * @throws IOException if we can not read the container logs.
      */
     public static void readAcontainerLogs(DataInputStream valueStream,
         Writer writer) throws IOException {
@@ -849,10 +850,11 @@ public class AggregatedLogFormat {
      * Keep calling this till you get a {@link EOFException} for getting logs of
      * all types for a single container.
      * 
-     * @param valueStream
-     * @param out
-     * @param logUploadedTime
-     * @throws IOException
+     * @param valueStream the value stream
+     * @param out the print stream
+     * @param logUploadedTime the time stamp
+     * @throws IOException if we can not read the container log by specifying
+     * the container log type.
      */
     public static void readAContainerLogsForALogType(
         DataInputStream valueStream, PrintStream out, long logUploadedTime)
@@ -864,11 +866,11 @@ public class AggregatedLogFormat {
      * Keep calling this till you get a {@link EOFException} for getting logs of
      * all types for a single container for the specific bytes.
      *
-     * @param valueStream
-     * @param out
-     * @param logUploadedTime
-     * @param bytes
-     * @throws IOException
+     * @param valueStream the value stream
+     * @param out the output print stream
+     * @param logUploadedTime the log upload time stamp
+     * @param bytes the output size of the log
+     * @throws IOException if we can not read the container log
      */
     public static void readAContainerLogsForALogType(
         DataInputStream valueStream, PrintStream out, long logUploadedTime,
@@ -880,9 +882,9 @@ public class AggregatedLogFormat {
      * Keep calling this till you get a {@link EOFException} for getting logs of
      * all types for a single container.
      * 
-     * @param valueStream
-     * @param out
-     * @throws IOException
+     * @param valueStream the value stream
+     * @param out the output print stream
+     * @throws IOException if we can not read the container log
      */
     public static void readAContainerLogsForALogType(
         DataInputStream valueStream, PrintStream out)
@@ -893,11 +895,11 @@ public class AggregatedLogFormat {
     /**
      * Keep calling this till you get a {@link EOFException} for getting logs of
      * the specific types for a single container.
-     * @param valueStream
-     * @param out
-     * @param logUploadedTime
-     * @param logType
-     * @throws IOException
+     * @param valueStream the value stream
+     * @param out the output print stream
+     * @param logUploadedTime the log uploaded time stamp
+     * @param logType the given log type
+     * @throws IOException if we can not read the container logs
      */
     public static int readContainerLogsForALogType(
         DataInputStream valueStream, PrintStream out, long logUploadedTime,
@@ -909,11 +911,11 @@ public class AggregatedLogFormat {
     /**
      * Keep calling this till you get a {@link EOFException} for getting logs of
      * the specific types for a single container.
-     * @param valueStream
-     * @param out
-     * @param logUploadedTime
-     * @param logType
-     * @throws IOException
+     * @param valueStream the value stream
+     * @param out the output print stream
+     * @param logUploadedTime the log uploaded time stamp
+     * @param logType the given log type
+     * @throws IOException if we can not read the container logs
      */
     public static int readContainerLogsForALogType(
         DataInputStream valueStream, PrintStream out, long logUploadedTime,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/63231a2a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/LogAggregationUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/LogAggregationUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/LogAggregationUtils.java
index e8a28de..6d04c29 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/LogAggregationUtils.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/LogAggregationUtils.java
@@ -38,11 +38,11 @@ public class LogAggregationUtils {
 
   /**
    * Constructs the full filename for an application's log file per node.
-   * @param remoteRootLogDir
-   * @param appId
-   * @param user
-   * @param nodeId
-   * @param suffix
+   * @param remoteRootLogDir the aggregated remote root log dir
+   * @param appId the application Id
+   * @param user the application owner
+   * @param nodeId the node id
+   * @param suffix the log dir suffix
    * @return the remote log file.
    */
   public static Path getRemoteNodeLogFileForApp(Path remoteRootLogDir,
@@ -53,10 +53,10 @@ public class LogAggregationUtils {
 
   /**
    * Gets the remote app log dir.
-   * @param remoteRootLogDir
-   * @param appId
-   * @param user
-   * @param suffix
+   * @param remoteRootLogDir the aggregated log remote root log dir
+   * @param appId the application id
+   * @param user the application owner
+   * @param suffix the log directory suffix
    * @return the remote application specific log dir.
    */
   public static Path getRemoteAppLogDir(Path remoteRootLogDir,
@@ -67,9 +67,9 @@ public class LogAggregationUtils {
 
   /**
    * Gets the remote suffixed log dir for the user.
-   * @param remoteRootLogDir
-   * @param user
-   * @param suffix
+   * @param remoteRootLogDir the aggregated log remote root log dir
+   * @param user the application owner
+   * @param suffix the log dir suffix
    * @return the remote suffixed log dir.
    */
   public static Path getRemoteLogSuffixedDir(Path remoteRootLogDir,
@@ -83,8 +83,8 @@ public class LogAggregationUtils {
 
   /**
    * Gets the remote log user dir.
-   * @param remoteRootLogDir
-   * @param user
+   * @param remoteRootLogDir the aggregated log remote root log dir
+   * @param user the application owner
    * @return the remote per user log dir.
    */
   public static Path getRemoteLogUserDir(Path remoteRootLogDir, String user) {
@@ -93,7 +93,7 @@ public class LogAggregationUtils {
 
   /**
    * Returns the suffix component of the log dir.
-   * @param conf
+   * @param conf the configuration
    * @return the suffix which will be appended to the user log dir.
    */
   public static String getRemoteNodeLogDirSuffix(Configuration conf) {
@@ -104,7 +104,7 @@ public class LogAggregationUtils {
   
   /**
    * Converts a nodeId to a form used in the app log file name.
-   * @param nodeId
+   * @param nodeId the nodeId
    * @return the node string to be used to construct the file name.
    */
   @VisibleForTesting

http://git-wip-us.apache.org/repos/asf/hadoop/blob/63231a2a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/LogCLIHelpers.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/LogCLIHelpers.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/LogCLIHelpers.java
index 03acb33..0068eae 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/LogCLIHelpers.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/LogCLIHelpers.java
@@ -128,8 +128,17 @@ public class LogCLIHelpers implements Configurable {
   @VisibleForTesting
   public int dumpAContainerLogsForLogType(ContainerLogsRequest options,
       boolean outputFailure) throws IOException {
-    boolean foundAnyLogs = this.getFileController(options.getAppId(),
-        options.getAppOwner()).readAggregatedLogs(options, null);
+    LogAggregationFileController fc = null;
+    try {
+      fc = this.getFileController(
+          options.getAppId(), options.getAppOwner());
+    } catch (IOException ex) {
+      System.err.println(ex);
+    }
+    boolean foundAnyLogs = false;
+    if (fc != null) {
+      foundAnyLogs = fc.readAggregatedLogs(options, null);
+    }
     if (!foundAnyLogs) {
       if (outputFailure) {
         containerLogNotFound(options.getContainerId());
@@ -142,9 +151,17 @@ public class LogCLIHelpers implements Configurable {
   @Private
   public int dumpAContainerLogsForLogTypeWithoutNodeId(
       ContainerLogsRequest options) throws IOException {
-    boolean foundAnyLogs = getFileController(options.getAppId(),
-        options.getAppOwner()).readAggregatedLogs(
-        options, null);
+    LogAggregationFileController fc = null;
+    try {
+      fc = this.getFileController(
+          options.getAppId(), options.getAppOwner());
+    } catch (IOException ex) {
+      System.err.println(ex);
+    }
+    boolean foundAnyLogs = false;
+    if (fc != null) {
+      foundAnyLogs = fc.readAggregatedLogs(options, null);
+    }
     if (!foundAnyLogs) {
       containerLogNotFound(options.getContainerId());
       return -1;
@@ -155,9 +172,17 @@ public class LogCLIHelpers implements Configurable {
   @Private
   public int dumpAllContainersLogs(ContainerLogsRequest options)
       throws IOException {
-    boolean foundAnyLogs = getFileController(options.getAppId(),
-        options.getAppOwner()).readAggregatedLogs(
-        options, null);
+    LogAggregationFileController fc = null;
+    try {
+      fc = this.getFileController(
+          options.getAppId(), options.getAppOwner());
+    } catch (IOException ex) {
+      System.err.println(ex);
+    }
+    boolean foundAnyLogs = false;
+    if (fc != null) {
+      foundAnyLogs = fc.readAggregatedLogs(options, null);
+    }
     if (!foundAnyLogs) {
       emptyLogDir(LogAggregationUtils.getRemoteAppLogDir(
           conf, options.getAppId(), options.getAppOwner())

http://git-wip-us.apache.org/repos/asf/hadoop/blob/63231a2a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/filecontroller/LogAggregationFileController.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/filecontroller/LogAggregationFileController.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/filecontroller/LogAggregationFileController.java
index 87344a7..aafdb66 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/filecontroller/LogAggregationFileController.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/filecontroller/LogAggregationFileController.java
@@ -201,6 +201,7 @@ public abstract class LogAggregationFileController {
    * Output container log.
    * @param logRequest {@link ContainerLogsRequest}
    * @param os the output stream
+   * @return true if we can read the aggregated logs successfully
    * @throws IOException if we can not access the log file.
    */
   public abstract boolean readAggregatedLogs(ContainerLogsRequest logRequest,
@@ -228,9 +229,9 @@ public abstract class LogAggregationFileController {
   /**
    * Returns the owner of the application.
    *
-   * @param the aggregatedLog path.
-   * @return the application owner.
-   * @throws IOException
+   * @param aggregatedLogPath the aggregatedLog path
+   * @return the application owner
+   * @throws IOException if we can not get the application owner
    */
   public abstract String getApplicationOwner(Path aggregatedLogPath)
       throws IOException;
@@ -239,9 +240,9 @@ public abstract class LogAggregationFileController {
    * Returns ACLs for the application. An empty map is returned if no ACLs are
    * found.
    *
-   * @param the aggregatedLog path.
+   * @param aggregatedLogPath the aggregatedLog path.
    * @return a map of the Application ACLs.
-   * @throws IOException
+   * @throws IOException if we can not get the application acls
    */
   public abstract Map<ApplicationAccessType, String> getApplicationAcls(
       Path aggregatedLogPath) throws IOException;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/63231a2a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/filecontroller/tfile/LogAggregationTFileController.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/filecontroller/tfile/LogAggregationTFileController.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/filecontroller/tfile/LogAggregationTFileController.java
index d2038e2..aaed538 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/filecontroller/tfile/LogAggregationTFileController.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/filecontroller/tfile/LogAggregationTFileController.java
@@ -172,7 +172,6 @@ public class LogAggregationTFileController
     byte[] buf = new byte[65535];
     while (nodeFiles != null && nodeFiles.hasNext()) {
       final FileStatus thisNodeFile = nodeFiles.next();
-      LOG.error(thisNodeFile.getPath().toString());
       String nodeName = thisNodeFile.getPath().getName();
       if (nodeName.equals(appId + ".har")) {
         Path p = new Path("har:///"


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org