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 vi...@apache.org on 2014/12/08 10:15:24 UTC

[01/29] hadoop git commit: YARN-1156. Enhance NodeManager AllocatedGB and AvailableGB metrics for aggregation of decimal values. (Contributed by Tsuyoshi OZAWA)

Repository: hadoop
Updated Branches:
  refs/heads/HDFS-EC bdc01015c -> 833185345


YARN-1156. Enhance NodeManager AllocatedGB and AvailableGB metrics for aggregation of decimal values. (Contributed by Tsuyoshi OZAWA)


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

Branch: refs/heads/HDFS-EC
Commit: e65b7c5ff6b0c013e510e750fe5cf59acfefea5f
Parents: 7caa3bc
Author: Junping Du <ju...@apache.org>
Authored: Wed Dec 3 04:11:18 2014 -0800
Committer: Junping Du <ju...@apache.org>
Committed: Wed Dec 3 04:11:18 2014 -0800

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                  |  3 +++
 .../nodemanager/metrics/NodeManagerMetrics.java  | 19 ++++++++++++++-----
 .../metrics/TestNodeManagerMetrics.java          | 17 ++++++++++++-----
 3 files changed, 29 insertions(+), 10 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/e65b7c5f/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index 1e336b7..421e5ea 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -112,6 +112,9 @@ Release 2.7.0 - UNRELEASED
     YARN-2136. Changed RMStateStore to ignore store opearations when fenced.
     (Varun Saxena via jianhe)
 
+    YARN-1156. Enhance NodeManager AllocatedGB and AvailableGB metrics 
+    for aggregation of decimal values. (Tsuyoshi OZAWA via junping_du)
+
   OPTIMIZATIONS
 
   BUG FIXES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e65b7c5f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/metrics/NodeManagerMetrics.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/metrics/NodeManagerMetrics.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/metrics/NodeManagerMetrics.java
index a3637d5..beaafe1 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/metrics/NodeManagerMetrics.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/metrics/NodeManagerMetrics.java
@@ -47,6 +47,9 @@ public class NodeManagerMetrics {
   @Metric("Container launch duration")
       MutableRate containerLaunchDuration;
 
+  private long allocatedMB;
+  private long availableMB;
+
   public static NodeManagerMetrics create() {
     return create(DefaultMetricsSystem.instance());
   }
@@ -92,22 +95,27 @@ public class NodeManagerMetrics {
 
   public void allocateContainer(Resource res) {
     allocatedContainers.incr();
-    allocatedGB.incr(res.getMemory() / 1024);
-    availableGB.decr(res.getMemory() / 1024);
+    allocatedMB = allocatedMB + res.getMemory();
+    allocatedGB.set((int)Math.ceil(allocatedMB/1024d));
+    availableMB = availableMB - res.getMemory();
+    availableGB.set((int)Math.floor(availableMB/1024d));
     allocatedVCores.incr(res.getVirtualCores());
     availableVCores.decr(res.getVirtualCores());
   }
 
   public void releaseContainer(Resource res) {
     allocatedContainers.decr();
-    allocatedGB.decr(res.getMemory() / 1024);
-    availableGB.incr(res.getMemory() / 1024);
+    allocatedMB = allocatedMB - res.getMemory();
+    allocatedGB.set((int)Math.ceil(allocatedMB/1024d));
+    availableMB = availableMB + res.getMemory();
+    availableGB.set((int)Math.floor(availableMB/1024d));
     allocatedVCores.decr(res.getVirtualCores());
     availableVCores.incr(res.getVirtualCores());
   }
 
   public void addResource(Resource res) {
-    availableGB.incr(res.getMemory() / 1024);
+    availableMB = availableMB + res.getMemory();
+    availableGB.incr((int)Math.floor(availableMB/1024d));
     availableVCores.incr(res.getVirtualCores());
   }
 
@@ -118,4 +126,5 @@ public class NodeManagerMetrics {
   public int getRunningContainers() {
     return containersRunning.value();
   }
+
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e65b7c5f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/metrics/TestNodeManagerMetrics.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/metrics/TestNodeManagerMetrics.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/metrics/TestNodeManagerMetrics.java
index 9906e36..d2a0691 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/metrics/TestNodeManagerMetrics.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/metrics/TestNodeManagerMetrics.java
@@ -33,13 +33,14 @@ public class TestNodeManagerMetrics {
     total.setMemory(8*GiB);
     total.setVirtualCores(16);
     Resource resource = Records.newRecord(Resource.class);
-    resource.setMemory(1*GiB);
+    resource.setMemory(512); //512MiB
     resource.setVirtualCores(2);
 
 
     metrics.addResource(total);
 
-    for (int i = 5; i-- > 0;) {
+    for (int i = 10; i-- > 0;) {
+      // allocate 10 containers(allocatedGB: 5GiB, availableGB: 3GiB)
       metrics.launchedContainer();
       metrics.allocateContainer(resource);
     }
@@ -48,6 +49,7 @@ public class TestNodeManagerMetrics {
     metrics.endInitingContainer();
     metrics.runningContainer();
     metrics.endRunningContainer();
+    // Releasing 3 containers(allocatedGB: 3.5GiB, availableGB: 4.5GiB)
     metrics.completedContainer();
     metrics.releaseContainer(resource);
 
@@ -61,12 +63,17 @@ public class TestNodeManagerMetrics {
     metrics.runningContainer();
     metrics.addContainerLaunchDuration(1);
 
-    checkMetrics(5, 1, 1, 1, 1, 1, 2, 2, 6, 4, 12);
+    // availableGB is expected to be floored,
+    // while allocatedGB is expected to be ceiled.
+    // allocatedGB: 3.5GB allocated memory is shown as 4GB
+    // availableGB: 4.5GB available memory is shown as 4GB
+    checkMetrics(10, 1, 1, 1, 1, 1, 4, 7, 4, 14, 2);
   }
 
   private void checkMetrics(int launched, int completed, int failed, int killed,
-                            int initing, int running, int allocatedGB,
-                            int allocatedContainers, int availableGB, int allocatedVCores, int availableVCores) {
+      int initing, int running, int allocatedGB,
+      int allocatedContainers, int availableGB, int allocatedVCores,
+      int availableVCores) {
     MetricsRecordBuilder rb = getMetrics("NodeManagerMetrics");
     assertCounter("ContainersLaunched", launched, rb);
     assertCounter("ContainersCompleted", completed, rb);


[21/29] hadoop git commit: YARN-2056. Disable preemption at Queue level. Contributed by Eric Payne

Posted by vi...@apache.org.
YARN-2056. Disable preemption at Queue level. Contributed by Eric Payne


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

Branch: refs/heads/HDFS-EC
Commit: 4b130821995a3cfe20c71e38e0f63294085c0491
Parents: 3c72f54
Author: Jason Lowe <jl...@apache.org>
Authored: Fri Dec 5 21:06:48 2014 +0000
Committer: Jason Lowe <jl...@apache.org>
Committed: Fri Dec 5 21:06:48 2014 +0000

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                 |   2 +
 .../ProportionalCapacityPreemptionPolicy.java   | 170 +++++++++--
 ...estProportionalCapacityPreemptionPolicy.java | 283 ++++++++++++++++++-
 3 files changed, 424 insertions(+), 31 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/4b130821/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index 9804d61..0b88959 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -126,6 +126,8 @@ Release 2.7.0 - UNRELEASED
 
     YARN-2301. Improved yarn container command. (Naganarasimha G R via jianhe)
 
+    YARN-2056. Disable preemption at Queue level (Eric Payne via jlowe)
+
   OPTIMIZATIONS
 
   BUG FIXES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4b130821/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/ProportionalCapacityPreemptionPolicy.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/ProportionalCapacityPreemptionPolicy.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/ProportionalCapacityPreemptionPolicy.java
index 0f48b0c..1a3f804 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/ProportionalCapacityPreemptionPolicy.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/ProportionalCapacityPreemptionPolicy.java
@@ -27,6 +27,7 @@ import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 import java.util.NavigableSet;
+import java.util.PriorityQueue;
 import java.util.Set;
 
 import org.apache.commons.logging.Log;
@@ -111,6 +112,9 @@ public class ProportionalCapacityPreemptionPolicy implements SchedulingEditPolic
   public static final String NATURAL_TERMINATION_FACTOR =
       "yarn.resourcemanager.monitor.capacity.preemption.natural_termination_factor";
 
+  public static final String BASE_YARN_RM_PREEMPTION = "yarn.scheduler.capacity.";
+  public static final String SUFFIX_DISABLE_PREEMPTION = ".disable_preemption";
+
   // the dispatcher to send preempt and kill events
   public EventHandler<ContainerPreemptEvent> dispatcher;
 
@@ -192,7 +196,7 @@ public class ProportionalCapacityPreemptionPolicy implements SchedulingEditPolic
     // extract a summary of the queues from scheduler
     TempQueue tRoot;
     synchronized (scheduler) {
-      tRoot = cloneQueues(root, clusterResources);
+      tRoot = cloneQueues(root, clusterResources, false);
     }
 
     // compute the ideal distribution of resources among queues
@@ -370,28 +374,60 @@ public class ProportionalCapacityPreemptionPolicy implements SchedulingEditPolic
   private void computeFixpointAllocation(ResourceCalculator rc,
       Resource tot_guarant, Collection<TempQueue> qAlloc, Resource unassigned, 
       boolean ignoreGuarantee) {
+    // Prior to assigning the unused resources, process each queue as follows:
+    // If current > guaranteed, idealAssigned = guaranteed + untouchable extra
+    // Else idealAssigned = current;
+    // Subtract idealAssigned resources from unassigned.
+    // If the queue has all of its needs met (that is, if 
+    // idealAssigned >= current + pending), remove it from consideration.
+    // Sort queues from most under-guaranteed to most over-guaranteed.
+    TQComparator tqComparator = new TQComparator(rc, tot_guarant);
+    PriorityQueue<TempQueue> orderedByNeed =
+                                 new PriorityQueue<TempQueue>(10,tqComparator);
+    for (Iterator<TempQueue> i = qAlloc.iterator(); i.hasNext();) {
+      TempQueue q = i.next();
+      if (Resources.greaterThan(rc, tot_guarant, q.current, q.guaranteed)) {
+        q.idealAssigned = Resources.add(q.guaranteed, q.untouchableExtra);
+      } else {
+        q.idealAssigned = Resources.clone(q.current);
+      }
+      Resources.subtractFrom(unassigned, q.idealAssigned);
+      // If idealAssigned < (current + pending), q needs more resources, so
+      // add it to the list of underserved queues, ordered by need.
+      Resource curPlusPend = Resources.add(q.current, q.pending);
+      if (Resources.lessThan(rc, tot_guarant, q.idealAssigned, curPlusPend)) {
+        orderedByNeed.add(q);
+      }
+    }
+
     //assign all cluster resources until no more demand, or no resources are left
-    while (!qAlloc.isEmpty() && Resources.greaterThan(rc, tot_guarant,
-          unassigned, Resources.none())) {
+    while (!orderedByNeed.isEmpty()
+       && Resources.greaterThan(rc,tot_guarant, unassigned,Resources.none())) {
       Resource wQassigned = Resource.newInstance(0, 0);
-
       // we compute normalizedGuarantees capacity based on currently active
       // queues
-      resetCapacity(rc, unassigned, qAlloc, ignoreGuarantee);
-      
-      // offer for each queue their capacity first and in following invocations
-      // their share of over-capacity
-      for (Iterator<TempQueue> i = qAlloc.iterator(); i.hasNext();) {
+      resetCapacity(rc, unassigned, orderedByNeed, ignoreGuarantee);
+
+      // For each underserved queue (or set of queues if multiple are equally
+      // underserved), offer its share of the unassigned resources based on its
+      // normalized guarantee. After the offer, if the queue is not satisfied,
+      // place it back in the ordered list of queues, recalculating its place
+      // in the order of most under-guaranteed to most over-guaranteed. In this
+      // way, the most underserved queue(s) are always given resources first.
+      Collection<TempQueue> underserved =
+          getMostUnderservedQueues(orderedByNeed, tqComparator);
+      for (Iterator<TempQueue> i = underserved.iterator(); i.hasNext();) {
         TempQueue sub = i.next();
-        Resource wQavail =
-          Resources.multiply(unassigned, sub.normalizedGuarantee);
+        Resource wQavail = Resources.multiplyAndNormalizeUp(rc,
+            unassigned, sub.normalizedGuarantee, Resource.newInstance(1, 1));
         Resource wQidle = sub.offer(wQavail, rc, tot_guarant);
         Resource wQdone = Resources.subtract(wQavail, wQidle);
-        // if the queue returned a value > 0 it means it is fully satisfied
-        // and it is removed from the list of active queues qAlloc
-        if (!Resources.greaterThan(rc, tot_guarant,
+
+        if (Resources.greaterThan(rc, tot_guarant,
               wQdone, Resources.none())) {
-          i.remove();
+          // The queue is still asking for more. Put it back in the priority
+          // queue, recalculating its order based on need.
+          orderedByNeed.add(sub);
         }
         Resources.addTo(wQassigned, wQdone);
       }
@@ -399,6 +435,27 @@ public class ProportionalCapacityPreemptionPolicy implements SchedulingEditPolic
     }
   }
 
+  // Take the most underserved TempQueue (the one on the head). Collect and
+  // return the list of all queues that have the same idealAssigned
+  // percentage of guaranteed.
+  protected Collection<TempQueue> getMostUnderservedQueues(
+      PriorityQueue<TempQueue> orderedByNeed, TQComparator tqComparator) {
+    ArrayList<TempQueue> underserved = new ArrayList<TempQueue>();
+    while (!orderedByNeed.isEmpty()) {
+      TempQueue q1 = orderedByNeed.remove();
+      underserved.add(q1);
+      TempQueue q2 = orderedByNeed.peek();
+      // q1's pct of guaranteed won't be larger than q2's. If it's less, then
+      // return what has already been collected. Otherwise, q1's pct of
+      // guaranteed == that of q2, so add q2 to underserved list during the
+      // next pass.
+      if (q2 == null || tqComparator.compare(q1,q2) < 0) {
+        return underserved;
+      }
+    }
+    return underserved;
+  }
+
   /**
    * Computes a normalizedGuaranteed capacity based on active queues
    * @param rc resource calculator
@@ -626,9 +683,11 @@ public class ProportionalCapacityPreemptionPolicy implements SchedulingEditPolic
    *
    * @param root the root of the CapacityScheduler queue hierarchy
    * @param clusterResources the total amount of resources in the cluster
+   * @param parentDisablePreempt true if disable preemption is set for parent
    * @return the root of the cloned queue hierarchy
    */
-  private TempQueue cloneQueues(CSQueue root, Resource clusterResources) {
+  private TempQueue cloneQueues(CSQueue root, Resource clusterResources,
+      boolean parentDisablePreempt) {
     TempQueue ret;
     synchronized (root) {
       String queueName = root.getQueueName();
@@ -639,19 +698,46 @@ public class ProportionalCapacityPreemptionPolicy implements SchedulingEditPolic
       Resource current = Resources.multiply(clusterResources, absUsed);
       Resource guaranteed = Resources.multiply(clusterResources, absCap);
       Resource maxCapacity = Resources.multiply(clusterResources, absMaxCap);
+
+      boolean queueDisablePreemption = false;
+      String queuePropName = BASE_YARN_RM_PREEMPTION + root.getQueuePath()
+                               + SUFFIX_DISABLE_PREEMPTION;
+      queueDisablePreemption = scheduler.getConfiguration()
+                              .getBoolean(queuePropName, parentDisablePreempt);
+
+      Resource extra = Resource.newInstance(0, 0);
+      if (Resources.greaterThan(rc, clusterResources, current, guaranteed)) {
+        extra = Resources.subtract(current, guaranteed);
+      }
       if (root instanceof LeafQueue) {
         LeafQueue l = (LeafQueue) root;
         Resource pending = l.getTotalResourcePending();
         ret = new TempQueue(queueName, current, pending, guaranteed,
             maxCapacity);
-
+        if (queueDisablePreemption) {
+          ret.untouchableExtra = extra;
+        } else {
+          ret.preemptableExtra = extra;
+        }
         ret.setLeafQueue(l);
       } else {
         Resource pending = Resource.newInstance(0, 0);
         ret = new TempQueue(root.getQueueName(), current, pending, guaranteed,
             maxCapacity);
+        Resource childrensPreemptable = Resource.newInstance(0, 0);
         for (CSQueue c : root.getChildQueues()) {
-          ret.addChild(cloneQueues(c, clusterResources));
+          TempQueue subq =
+                cloneQueues(c, clusterResources, queueDisablePreemption);
+          Resources.addTo(childrensPreemptable, subq.preemptableExtra);
+          ret.addChild(subq);
+        }
+        // untouchableExtra = max(extra - childrenPreemptable, 0)
+        if (Resources.greaterThanOrEqual(
+              rc, clusterResources, childrensPreemptable, extra)) {
+          ret.untouchableExtra = Resource.newInstance(0, 0);
+        } else {
+          ret.untouchableExtra =
+                Resources.subtractFrom(extra, childrensPreemptable);
         }
       }
     }
@@ -690,6 +776,8 @@ public class ProportionalCapacityPreemptionPolicy implements SchedulingEditPolic
     Resource idealAssigned;
     Resource toBePreempted;
     Resource actuallyPreempted;
+    Resource untouchableExtra;
+    Resource preemptableExtra;
 
     double normalizedGuarantee;
 
@@ -708,6 +796,8 @@ public class ProportionalCapacityPreemptionPolicy implements SchedulingEditPolic
       this.toBePreempted = Resource.newInstance(0, 0);
       this.normalizedGuarantee = Float.NaN;
       this.children = new ArrayList<TempQueue>();
+      this.untouchableExtra = Resource.newInstance(0, 0);
+      this.preemptableExtra = Resource.newInstance(0, 0);
     }
 
     public void setLeafQueue(LeafQueue l){
@@ -761,10 +851,20 @@ public class ProportionalCapacityPreemptionPolicy implements SchedulingEditPolic
         .append(" IDEAL_ASSIGNED: ").append(idealAssigned)
         .append(" IDEAL_PREEMPT: ").append(toBePreempted)
         .append(" ACTUAL_PREEMPT: ").append(actuallyPreempted)
+        .append(" UNTOUCHABLE: ").append(untouchableExtra)
+        .append(" PREEMPTABLE: ").append(preemptableExtra)
         .append("\n");
 
       return sb.toString();
     }
+
+    public void printAll() {
+      LOG.info(this.toString());
+      for (TempQueue sub : this.getChildren()) {
+        sub.printAll();
+      }
+    }
+
     public void assignPreemption(float scalingFactor,
         ResourceCalculator rc, Resource clusterResource) {
       if (Resources.greaterThan(rc, clusterResource, current, idealAssigned)) {
@@ -793,4 +893,38 @@ public class ProportionalCapacityPreemptionPolicy implements SchedulingEditPolic
 
   }
 
+  static class TQComparator implements Comparator<TempQueue> {
+    private ResourceCalculator rc;
+    private Resource clusterRes;
+
+    TQComparator(ResourceCalculator rc, Resource clusterRes) {
+      this.rc = rc;
+      this.clusterRes = clusterRes;
+    }
+
+    @Override
+    public int compare(TempQueue tq1, TempQueue tq2) {
+      if (getIdealPctOfGuaranteed(tq1) < getIdealPctOfGuaranteed(tq2)) {
+        return -1;
+      }
+      if (getIdealPctOfGuaranteed(tq1) > getIdealPctOfGuaranteed(tq2)) {
+        return 1;
+      }
+      return 0;
+    }
+
+    // Calculates idealAssigned / guaranteed
+    // TempQueues with 0 guarantees are always considered the most over
+    // capacity and therefore considered last for resources.
+    private double getIdealPctOfGuaranteed(TempQueue q) {
+      double pctOver = Integer.MAX_VALUE;
+      if (q != null && Resources.greaterThan(
+          rc, clusterRes, q.guaranteed, Resources.none())) {
+        pctOver =
+            Resources.divide(rc, clusterRes, q.idealAssigned, q.guaranteed);
+      }
+      return (pctOver);
+    }
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4b130821/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/TestProportionalCapacityPreemptionPolicy.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/TestProportionalCapacityPreemptionPolicy.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/TestProportionalCapacityPreemptionPolicy.java
index 24e70bb..ca67ef0 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/TestProportionalCapacityPreemptionPolicy.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/TestProportionalCapacityPreemptionPolicy.java
@@ -17,16 +17,19 @@
  */
 package org.apache.hadoop.yarn.server.resourcemanager.monitor.capacity;
 
+import static org.apache.hadoop.yarn.server.resourcemanager.monitor.capacity.ProportionalCapacityPreemptionPolicy.BASE_YARN_RM_PREEMPTION;
 import static org.apache.hadoop.yarn.server.resourcemanager.monitor.capacity.ProportionalCapacityPreemptionPolicy.MAX_IGNORED_OVER_CAPACITY;
 import static org.apache.hadoop.yarn.server.resourcemanager.monitor.capacity.ProportionalCapacityPreemptionPolicy.MONITORING_INTERVAL;
 import static org.apache.hadoop.yarn.server.resourcemanager.monitor.capacity.ProportionalCapacityPreemptionPolicy.NATURAL_TERMINATION_FACTOR;
 import static org.apache.hadoop.yarn.server.resourcemanager.monitor.capacity.ProportionalCapacityPreemptionPolicy.OBSERVE_ONLY;
+import static org.apache.hadoop.yarn.server.resourcemanager.monitor.capacity.ProportionalCapacityPreemptionPolicy.SUFFIX_DISABLE_PREEMPTION;
 import static org.apache.hadoop.yarn.server.resourcemanager.monitor.capacity.ProportionalCapacityPreemptionPolicy.TOTAL_PREEMPTION_PER_ROUND;
 import static org.apache.hadoop.yarn.server.resourcemanager.monitor.capacity.ProportionalCapacityPreemptionPolicy.WAIT_TIME_BEFORE_KILL;
 import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.ContainerPreemptEventType.KILL_CONTAINER;
 import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.ContainerPreemptEventType.PREEMPT_CONTAINER;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 import static org.mockito.Matchers.argThat;
 import static org.mockito.Matchers.isA;
@@ -62,6 +65,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ContainerPreemptE
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ContainerPreemptEventType;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CSQueue;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.LeafQueue;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.ParentQueue;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerApp;
@@ -86,6 +90,7 @@ public class TestProportionalCapacityPreemptionPolicy {
   Clock mClock = null;
   Configuration conf = null;
   CapacityScheduler mCS = null;
+  CapacitySchedulerConfiguration schedConf = null;
   EventHandler<ContainerPreemptEvent> mDisp = null;
   ResourceCalculator rc = new DefaultResourceCalculator();
   final ApplicationAttemptId appA = ApplicationAttemptId.newInstance(
@@ -98,6 +103,8 @@ public class TestProportionalCapacityPreemptionPolicy {
       ApplicationId.newInstance(TS, 3), 0);
   final ApplicationAttemptId appE = ApplicationAttemptId.newInstance(
       ApplicationId.newInstance(TS, 4), 0);
+  final ApplicationAttemptId appF = ApplicationAttemptId.newInstance(
+      ApplicationId.newInstance(TS, 4), 0);
   final ArgumentCaptor<ContainerPreemptEvent> evtCaptor =
     ArgumentCaptor.forClass(ContainerPreemptEvent.class);
 
@@ -123,6 +130,8 @@ public class TestProportionalCapacityPreemptionPolicy {
     mClock = mock(Clock.class);
     mCS = mock(CapacityScheduler.class);
     when(mCS.getResourceCalculator()).thenReturn(rc);
+    schedConf = new CapacitySchedulerConfiguration();
+    when(mCS.getConfiguration()).thenReturn(schedConf);
     mDisp = mock(EventHandler.class);
     rand = new Random();
     long seed = rand.nextLong();
@@ -266,6 +275,240 @@ public class TestProportionalCapacityPreemptionPolicy {
   }
 
   @Test
+  public void testPerQueueDisablePreemption() {
+    int[][] qData = new int[][]{
+        //  /    A    B    C
+        { 100,  55,  25,  20 },  // abs
+        { 100, 100, 100, 100 },  // maxCap
+        { 100,   0,  54,  46 },  // used
+        {  10,  10,   0,   0 },  // pending
+        {   0,   0,   0,   0 },  // reserved
+       //     appA appB appC
+        {   3,   1,   1,   1 },  // apps
+        {  -1,   1,   1,   1 },  // req granularity
+        {   3,   0,   0,  0 },  // subqueues
+      };
+
+    schedConf.setBoolean(BASE_YARN_RM_PREEMPTION
+        + "root.queueB" + SUFFIX_DISABLE_PREEMPTION, true);
+
+    ProportionalCapacityPreemptionPolicy policy = buildPolicy(qData);
+    policy.editSchedule();
+    // With PREEMPTION_DISABLED set for queueB, get resources from queueC
+    verify(mDisp, times(10)).handle(argThat(new IsPreemptionRequestFor(appC)));
+    verify(mDisp, never()).handle(argThat(new IsPreemptionRequestFor(appB)));
+
+    // With no PREEMPTION_DISABLED set for queueB, resources will be preempted
+    // from both queueB and queueC. Test must be reset for so that the mDisp
+    // event handler will count only events from the following test and not the
+    // previous one.
+    setup();
+    ProportionalCapacityPreemptionPolicy policy2 = buildPolicy(qData);
+    
+    schedConf.setBoolean(BASE_YARN_RM_PREEMPTION
+        + "root.queueB" + SUFFIX_DISABLE_PREEMPTION, false);
+    policy2.editSchedule();
+
+    verify(mDisp, times(4)).handle(argThat(new IsPreemptionRequestFor(appB)));
+    verify(mDisp, times(6)).handle(argThat(new IsPreemptionRequestFor(appC)));
+  }
+
+  @Test
+  public void testPerQueueDisablePreemptionHierarchical() {
+    int[][] qData = new int[][] {
+      //  /    A              D
+      //            B    C         E    F
+      { 200, 100,  50,  50, 100,  10,  90 },  // abs
+      { 200, 200, 200, 200, 200, 200, 200 },  // maxCap
+      { 200, 110,  60,  50,  90,  90,   0 },  // used
+      {  10,   0,   0,   0,  10,   0,  10 },  // pending
+      {   0,   0,   0,   0,   0,   0,   0 },  // reserved
+      //          appA appB      appC appD
+      {   4,   2,   1,   1,   2,   1,   1 },  // apps
+      {  -1,  -1,   1,   1,  -1,   1,   1 },  // req granularity
+      {   2,   2,   0,   0,   2,   0,   0 },  // subqueues
+    };
+    ProportionalCapacityPreemptionPolicy policy = buildPolicy(qData);
+    policy.editSchedule();
+    // verify capacity taken from queueB (appA), not queueE (appC) despite 
+    // queueE being far over its absolute capacity because queueA (queueB's
+    // parent) is over capacity and queueD (queueE's parent) is not.
+    ApplicationAttemptId expectedAttemptOnQueueB = 
+        ApplicationAttemptId.newInstance(
+            appA.getApplicationId(), appA.getAttemptId());
+    assertTrue("appA should be running on queueB",
+        mCS.getAppsInQueue("queueB").contains(expectedAttemptOnQueueB));
+    verify(mDisp, times(9)).handle(argThat(new IsPreemptionRequestFor(appA)));
+
+    // Need to call setup() again to reset mDisp
+    setup();
+    // Disable preemption for queueB and it's children
+    schedConf.setBoolean(BASE_YARN_RM_PREEMPTION
+        + "root.queueA.queueB" + SUFFIX_DISABLE_PREEMPTION, true);
+    ProportionalCapacityPreemptionPolicy policy2 = buildPolicy(qData);
+    policy2.editSchedule();
+    ApplicationAttemptId expectedAttemptOnQueueC = 
+        ApplicationAttemptId.newInstance(
+            appB.getApplicationId(), appB.getAttemptId());
+    ApplicationAttemptId expectedAttemptOnQueueE = 
+        ApplicationAttemptId.newInstance(
+            appC.getApplicationId(), appC.getAttemptId());
+    // Now, all of queueB's (appA) over capacity is not preemptable, so neither
+    // is queueA's. Verify that capacity is taken from queueE (appC).
+    assertTrue("appB should be running on queueC",
+        mCS.getAppsInQueue("queueC").contains(expectedAttemptOnQueueC));
+    assertTrue("appC should be running on queueE",
+        mCS.getAppsInQueue("queueE").contains(expectedAttemptOnQueueE));
+    // Resources should have come from queueE (appC) and neither of queueA's
+    // children.
+    verify(mDisp, never()).handle(argThat(new IsPreemptionRequestFor(appA)));
+    verify(mDisp, times(9)).handle(argThat(new IsPreemptionRequestFor(appC)));
+  }
+
+  @Test
+  public void testPerQueueDisablePreemptionBroadHierarchical() {
+    int[][] qData = new int[][] {
+        //  /    A              D              G    
+        //            B    C         E    F         H    I
+        {1000, 350, 150, 200, 400, 200, 200, 250, 100, 150 },  // abs
+        {1000,1000,1000,1000,1000,1000,1000,1000,1000,1000 },  // maxCap
+        {1000, 400, 200, 200, 400, 250, 150, 200, 150,  50 },  // used
+        {  50,   0,   0,   0,  50,   0,  50,   0,   0,   0 },  // pending
+        {   0,   0,   0,   0,   0,   0,   0,   0,   0,   0 },  // reserved
+        //          appA appB      appC appD      appE appF
+        {   6,   2,   1,   1,   2,   1,   1,   2,   1,   1 },  // apps
+        {  -1,  -1,   1,   1,  -1,   1,   1,  -1,   1,   1 },  // req granulrity
+        {   3,   2,   0,   0,   2,   0,   0,   2,   0,   0 },  // subqueues
+      };
+
+    ProportionalCapacityPreemptionPolicy policy = buildPolicy(qData);
+    policy.editSchedule();
+    // queueF(appD) wants resources, Verify that resources come from queueE(appC)
+    // because it's a sibling and queueB(appA) because queueA is over capacity.
+    verify(mDisp, times(28)).handle(argThat(new IsPreemptionRequestFor(appA)));
+    verify(mDisp, times(22)).handle(argThat(new IsPreemptionRequestFor(appC)));
+
+    // Need to call setup() again to reset mDisp
+    setup();
+    // Disable preemption for queueB(appA)
+    schedConf.setBoolean(BASE_YARN_RM_PREEMPTION
+        + "root.queueA.queueB" + SUFFIX_DISABLE_PREEMPTION, true);
+    ProportionalCapacityPreemptionPolicy policy2 = buildPolicy(qData);
+    policy2.editSchedule();
+    // Now that queueB(appA) is not preemptable, verify that resources come
+    // from queueE(appC)
+    verify(mDisp, times(50)).handle(argThat(new IsPreemptionRequestFor(appC)));
+    verify(mDisp, never()).handle(argThat(new IsPreemptionRequestFor(appA)));
+
+    setup();
+    // Disable preemption for two of the 3 queues with over-capacity.
+    schedConf.setBoolean(BASE_YARN_RM_PREEMPTION
+        + "root.queueD.queueE" + SUFFIX_DISABLE_PREEMPTION, true);
+    schedConf.setBoolean(BASE_YARN_RM_PREEMPTION
+        + "root.queueA.queueB" + SUFFIX_DISABLE_PREEMPTION, true);
+    ProportionalCapacityPreemptionPolicy policy3 = buildPolicy(qData);
+    policy3.editSchedule();
+
+    // Verify that the request was starved out even though queueH(appE) is
+    // over capacity. This is because queueG (queueH's parent) is NOT
+    // overcapacity.
+    verify(mDisp, never()).handle(argThat(new IsPreemptionRequestFor(appA))); // queueB
+    verify(mDisp, never()).handle(argThat(new IsPreemptionRequestFor(appB))); // queueC
+    verify(mDisp, never()).handle(argThat(new IsPreemptionRequestFor(appC))); // queueE
+    verify(mDisp, never()).handle(argThat(new IsPreemptionRequestFor(appE))); // queueH
+    verify(mDisp, never()).handle(argThat(new IsPreemptionRequestFor(appF))); // queueI
+  }
+
+  @Test
+  public void testPerQueueDisablePreemptionInheritParent() {
+    int[][] qData = new int[][] {
+        //  /    A                   E          
+        //            B    C    D         F    G    H
+        {1000, 500, 200, 200, 100, 500, 200, 200, 100 },  // abs (guar)
+        {1000,1000,1000,1000,1000,1000,1000,1000,1000 },  // maxCap
+        {1000, 700,   0, 350, 350, 300,   0, 200, 100 },  // used 
+        { 200,   0,   0,   0,   0, 200, 200,   0,   0 },  // pending
+        {   0,   0,   0,   0,   0,   0,   0,   0,   0 },  // reserved
+        //               appA appB      appC appD appE 
+        {   5,   2,   0,   1,   1,   3,   1,   1,   1 },  // apps 
+        {  -1,  -1,   1,   1,   1,  -1,   1,   1,   1 },  // req granulrity
+        {   2,   3,   0,   0,   0,   3,   0,   0,   0 },  // subqueues
+      };
+
+    ProportionalCapacityPreemptionPolicy policy = buildPolicy(qData);
+    policy.editSchedule();
+    // With all queues preemptable, resources should be taken from queueC(appA)
+    // and queueD(appB). Resources taken more from queueD(appB) than
+    // queueC(appA) because it's over its capacity by a larger percentage.
+    verify(mDisp, times(16)).handle(argThat(new IsPreemptionRequestFor(appA)));
+    verify(mDisp, times(182)).handle(argThat(new IsPreemptionRequestFor(appB)));
+
+    // Disable preemption for queueA and it's children. queueF(appC)'s request
+    // should starve.
+    setup(); // Call setup() to reset mDisp
+    schedConf.setBoolean(BASE_YARN_RM_PREEMPTION
+        + "root.queueA" + SUFFIX_DISABLE_PREEMPTION, true);
+    ProportionalCapacityPreemptionPolicy policy2 = buildPolicy(qData);
+    policy2.editSchedule();
+    verify(mDisp, never()).handle(argThat(new IsPreemptionRequestFor(appA))); // queueC
+    verify(mDisp, never()).handle(argThat(new IsPreemptionRequestFor(appB))); // queueD
+    verify(mDisp, never()).handle(argThat(new IsPreemptionRequestFor(appD))); // queueG
+    verify(mDisp, never()).handle(argThat(new IsPreemptionRequestFor(appE))); // queueH
+  }
+
+  @Test
+  public void testPerQueuePreemptionNotAllUntouchable() {
+    int[][] qData = new int[][] {
+      //  /      A                       E
+      //               B     C     D           F     G     H
+      { 2000, 1000,  800,  100,  100, 1000,  500,  300,  200 },  // abs
+      { 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000 },  // maxCap
+      { 2000, 1300,  300,  800,  200,  700,  500,    0,  200 },  // used
+      {  300,    0,    0,    0,    0,  300,    0,  300,    0 },  // pending
+      {    0,    0,    0,    0,    0,    0,    0,    0,    0 },  // reserved
+      //             appA  appB  appC        appD  appE  appF
+      {    6,    3,    1,    1,    1,    3,    1,    1,    1 },  // apps
+      {   -1,   -1,    1,    1,    1,   -1,    1,    1,    1 },  // req granularity
+      {    2,    3,    0,    0,    0,    3,    0,    0,    0 },  // subqueues
+    };
+    schedConf.setBoolean(BASE_YARN_RM_PREEMPTION
+        + "root.queueA.queueC" + SUFFIX_DISABLE_PREEMPTION, true);
+    ProportionalCapacityPreemptionPolicy policy = buildPolicy(qData);
+    policy.editSchedule();
+    // Although queueC(appB) is way over capacity and is untouchable,
+    // queueD(appC) is preemptable. Request should be filled from queueD(appC).
+    verify(mDisp, times(100)).handle(argThat(new IsPreemptionRequestFor(appC)));
+  }
+
+  @Test
+  public void testPerQueueDisablePreemptionRootDisablesAll() {
+    int[][] qData = new int[][] {
+        //  /    A              D              G    
+        //            B    C         E    F         H    I
+        {1000, 500, 250, 250, 250, 100, 150, 250, 100, 150 },  // abs
+        {1000,1000,1000,1000,1000,1000,1000,1000,1000,1000 },  // maxCap
+        {1000,  20,   0,  20, 490, 240, 250, 490, 240, 250 },  // used
+        { 200, 200, 200,   0,   0,   0,   0,   0,   0,   0 },  // pending
+        {   0,   0,   0,   0,   0,   0,   0,   0,   0,   0 },  // reserved
+        //          appA appB      appC appD      appE appF
+        {   6,   2,   1,   1,   2,   1,   1,   2,   1,   1 },  // apps
+        {  -1,  -1,   1,   1,  -1,   1,   1,  -1,   1,   1 },  // req granulrity
+        {   3,   2,   0,   0,   2,   0,   0,   2,   0,   0 },  // subqueues
+   };
+
+    ProportionalCapacityPreemptionPolicy policy = buildPolicy(qData);
+    schedConf.setBoolean(BASE_YARN_RM_PREEMPTION
+        + "root" + SUFFIX_DISABLE_PREEMPTION, true);
+    policy.editSchedule();
+    // All queues should be non-preemptable, so request should starve.
+    verify(mDisp, never()).handle(argThat(new IsPreemptionRequestFor(appB))); // queueC
+    verify(mDisp, never()).handle(argThat(new IsPreemptionRequestFor(appC))); // queueE
+    verify(mDisp, never()).handle(argThat(new IsPreemptionRequestFor(appD))); // queueB
+    verify(mDisp, never()).handle(argThat(new IsPreemptionRequestFor(appE))); // queueH
+    verify(mDisp, never()).handle(argThat(new IsPreemptionRequestFor(appF))); // queueI
+  }
+
+  @Test
   public void testOverCapacityImbalance() {
     int[][] qData = new int[][]{
       //  /   A   B   C
@@ -341,7 +584,7 @@ public class TestProportionalCapacityPreemptionPolicy {
     policy.editSchedule();
     // verify capacity taken from A1, not B1 despite B1 being far over
     // its absolute guaranteed capacity
-    verify(mDisp, times(10)).handle(argThat(new IsPreemptionRequestFor(appA)));
+    verify(mDisp, times(9)).handle(argThat(new IsPreemptionRequestFor(appA)));
   }
 
   @Test
@@ -390,15 +633,17 @@ public class TestProportionalCapacityPreemptionPolicy {
   @Test
   public void testHierarchicalLarge() {
     int[][] qData = new int[][] {
-      //  /    A   B   C    D   E   F    G   H   I
-      { 400, 200, 60, 140, 100, 70, 30, 100, 10, 90  },  // abs
-      { 400, 400, 400, 400, 400, 400, 400, 400, 400, 400, },  // maxCap
-      { 400, 210, 70,140, 100, 50, 50,  90, 90,  0  },  // used
-      {  10,   0,  0,  0,   0,  0,  0,   0,  0, 15  },  // pending
-      {   0,   0,  0,  0,   0,  0,  0,   0,  0,  0  },  // reserved
-      {   6,   2,  1,  1,   2,  1,  1,   2,  1,  1  },  // apps
-      {  -1,  -1,  1,  1,  -1,  1,  1,  -1,  1,  1  },  // req granularity
-      {   3,   2,  0,  0,   2,  0,  0,   2,  0,  0  },  // subqueues
+      //  /    A              D              G        
+      //            B    C         E    F         H    I
+      { 400, 200,  60, 140, 100,  70,  30, 100,  10,  90 },  // abs
+      { 400, 400, 400, 400, 400, 400, 400, 400, 400, 400 },  // maxCap
+      { 400, 210,  70, 140, 100,  50,  50,  90,  90,   0 },  // used
+      {  15,   0,   0,   0,   0,   0,   0,   0,   0,  15 },  // pending
+      {   0,   0,   0,   0,   0,   0,   0,   0,   0,   0 },  // reserved
+      //          appA appB      appC appD      appE appF
+      {   6,   2,   1,   1,   2,   1,   1,   2,   1,   1 },  // apps
+      {  -1,  -1,   1,   1,  -1,   1,   1,  -1,   1,   1 },  // req granularity
+      {   3,   2,   0,   0,   2,   0,   0,   2,   0,   0 },  // subqueues
     };
     ProportionalCapacityPreemptionPolicy policy = buildPolicy(qData);
     policy.editSchedule();
@@ -407,8 +652,8 @@ public class TestProportionalCapacityPreemptionPolicy {
 
     // XXX note: compensating for rounding error in Resources.multiplyTo
     // which is likely triggered since we use small numbers for readability
-    verify(mDisp, times(9)).handle(argThat(new IsPreemptionRequestFor(appA)));
-    verify(mDisp, times(4)).handle(argThat(new IsPreemptionRequestFor(appE)));
+    verify(mDisp, times(7)).handle(argThat(new IsPreemptionRequestFor(appA)));
+    verify(mDisp, times(5)).handle(argThat(new IsPreemptionRequestFor(appE)));
   }
 
   @Test
@@ -629,6 +874,7 @@ public class TestProportionalCapacityPreemptionPolicy {
     when(root.getAbsoluteUsedCapacity()).thenReturn(used[0] / tot);
     when(root.getAbsoluteCapacity()).thenReturn(abs[0] / tot);
     when(root.getAbsoluteMaximumCapacity()).thenReturn(maxCap[0] / tot);
+    when(root.getQueuePath()).thenReturn("root");
 
     for (int i = 1; i < queues.length; ++i) {
       final CSQueue q;
@@ -644,6 +890,10 @@ public class TestProportionalCapacityPreemptionPolicy {
       when(q.getAbsoluteUsedCapacity()).thenReturn(used[i] / tot);
       when(q.getAbsoluteCapacity()).thenReturn(abs[i] / tot);
       when(q.getAbsoluteMaximumCapacity()).thenReturn(maxCap[i] / tot);
+      String parentPathName = p.getQueuePath();
+      parentPathName = (parentPathName == null) ? "root" : parentPathName;
+      String queuePathName = (parentPathName+"."+queueName).replace("/","root");
+      when(q.getQueuePath()).thenReturn(queuePathName);
     }
     assert 0 == pqs.size();
     return root;
@@ -666,6 +916,8 @@ public class TestProportionalCapacityPreemptionPolicy {
   LeafQueue mockLeafQueue(ParentQueue p, float tot, int i, int[] abs, 
       int[] used, int[] pending, int[] reserved, int[] apps, int[] gran) {
     LeafQueue lq = mock(LeafQueue.class);
+    List<ApplicationAttemptId> appAttemptIdList = 
+        new ArrayList<ApplicationAttemptId>();
     when(lq.getTotalResourcePending()).thenReturn(
         Resource.newInstance(pending[i], 0));
     // consider moving where CapacityScheduler::comparator accessible
@@ -683,9 +935,14 @@ public class TestProportionalCapacityPreemptionPolicy {
       int aPending = pending[i] / apps[i];
       int aReserve = reserved[i] / apps[i];
       for (int a = 0; a < apps[i]; ++a) {
-        qApps.add(mockApp(i, appAlloc, aUsed, aPending, aReserve, gran[i]));
+        FiCaSchedulerApp mockFiCaApp =
+            mockApp(i, appAlloc, aUsed, aPending, aReserve, gran[i]);
+        qApps.add(mockFiCaApp);
         ++appAlloc;
+        appAttemptIdList.add(mockFiCaApp.getApplicationAttemptId());
       }
+      when(mCS.getAppsInQueue("queue" + (char)('A' + i - 1)))
+              .thenReturn(appAttemptIdList);
     }
     when(lq.getApplications()).thenReturn(qApps);
     if(setAMResourcePercent != 0.0f){


[15/29] hadoop git commit: HDFS-7454. Reduce memory footprint for AclEntries in NameNode. Contributed by Vinayakumar B.

Posted by vi...@apache.org.
HDFS-7454. Reduce memory footprint for AclEntries in NameNode. Contributed by Vinayakumar B.


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

Branch: refs/heads/HDFS-EC
Commit: 0653918dad855b394e8e3b8b3f512f474d872ee9
Parents: 7896815
Author: Haohui Mai <wh...@apache.org>
Authored: Thu Dec 4 20:49:45 2014 -0800
Committer: Haohui Mai <wh...@apache.org>
Committed: Thu Dec 4 20:49:45 2014 -0800

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |   3 +
 .../server/namenode/AclEntryStatusFormat.java   | 136 +++++++++++++++++++
 .../hadoop/hdfs/server/namenode/AclFeature.java |  24 +++-
 .../hadoop/hdfs/server/namenode/AclStorage.java |  30 +++-
 .../server/namenode/FSImageFormatPBINode.java   |  22 +--
 .../server/namenode/FSPermissionChecker.java    |  39 ++----
 .../snapshot/FSImageFormatPBSnapshot.java       |  13 +-
 .../hdfs/server/namenode/FSAclBaseTest.java     |   4 +-
 8 files changed, 223 insertions(+), 48 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/0653918d/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index 4432024..02f41cc 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -432,6 +432,9 @@ Release 2.7.0 - UNRELEASED
 
   OPTIMIZATIONS
 
+    HDFS-7454. Reduce memory footprint for AclEntries in NameNode.
+    (Vinayakumar B via wheat9)
+
   BUG FIXES
 
     HDFS-6741. Improve permission denied message when

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0653918d/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/AclEntryStatusFormat.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/AclEntryStatusFormat.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/AclEntryStatusFormat.java
new file mode 100644
index 0000000..82aa214
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/AclEntryStatusFormat.java
@@ -0,0 +1,136 @@
+/**
+ * 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 java.util.List;
+
+import org.apache.hadoop.fs.permission.AclEntry;
+import org.apache.hadoop.fs.permission.AclEntryScope;
+import org.apache.hadoop.fs.permission.AclEntryType;
+import org.apache.hadoop.fs.permission.FsAction;
+import org.apache.hadoop.hdfs.util.LongBitFormat;
+
+import com.google.common.collect.ImmutableList;
+
+/**
+ * Class to pack an AclEntry into an integer. <br>
+ * An ACL entry is represented by a 32-bit integer in Big Endian format. <br>
+ * The bits can be divided in four segments: <br>
+ * [0:1) || [1:3) || [3:6) || [6:7) || [7:32) <br>
+ * <br>
+ * [0:1) -- the scope of the entry (AclEntryScope) <br>
+ * [1:3) -- the type of the entry (AclEntryType) <br>
+ * [3:6) -- the permission of the entry (FsAction) <br>
+ * [6:7) -- A flag to indicate whether Named entry or not <br>
+ * [7:32) -- the name of the entry, which is an ID that points to a <br>
+ * string in the StringTableSection. <br>
+ */
+public enum AclEntryStatusFormat {
+
+  SCOPE(null, 1),
+  TYPE(SCOPE.BITS, 2),
+  PERMISSION(TYPE.BITS, 3),
+  NAMED_ENTRY_CHECK(PERMISSION.BITS, 1),
+  NAME(NAMED_ENTRY_CHECK.BITS, 25);
+
+  private final LongBitFormat BITS;
+
+  private AclEntryStatusFormat(LongBitFormat previous, int length) {
+    BITS = new LongBitFormat(name(), previous, length, 0);
+  }
+
+  static AclEntryScope getScope(int aclEntry) {
+    int ordinal = (int) SCOPE.BITS.retrieve(aclEntry);
+    return AclEntryScope.values()[ordinal];
+  }
+
+  static AclEntryType getType(int aclEntry) {
+    int ordinal = (int) TYPE.BITS.retrieve(aclEntry);
+    return AclEntryType.values()[ordinal];
+  }
+
+  static FsAction getPermission(int aclEntry) {
+    int ordinal = (int) PERMISSION.BITS.retrieve(aclEntry);
+    return FsAction.values()[ordinal];
+  }
+
+  static String getName(int aclEntry) {
+    int nameExists = (int) NAMED_ENTRY_CHECK.BITS.retrieve(aclEntry);
+    if (nameExists == 0) {
+      return null;
+    }
+    int id = (int) NAME.BITS.retrieve(aclEntry);
+    AclEntryType type = getType(aclEntry);
+    if (type == AclEntryType.USER) {
+      return SerialNumberManager.INSTANCE.getUser(id);
+    } else if (type == AclEntryType.GROUP) {
+      return SerialNumberManager.INSTANCE.getGroup(id);
+    }
+    return null;
+  }
+
+  static int toInt(AclEntry aclEntry) {
+    long aclEntryInt = 0;
+    aclEntryInt = SCOPE.BITS
+        .combine(aclEntry.getScope().ordinal(), aclEntryInt);
+    aclEntryInt = TYPE.BITS.combine(aclEntry.getType().ordinal(), aclEntryInt);
+    aclEntryInt = PERMISSION.BITS.combine(aclEntry.getPermission().ordinal(),
+        aclEntryInt);
+    if (aclEntry.getName() != null) {
+      aclEntryInt = NAMED_ENTRY_CHECK.BITS.combine(1, aclEntryInt);
+      if (aclEntry.getType() == AclEntryType.USER) {
+        int userId = SerialNumberManager.INSTANCE.getUserSerialNumber(aclEntry
+            .getName());
+        aclEntryInt = NAME.BITS.combine(userId, aclEntryInt);
+      } else if (aclEntry.getType() == AclEntryType.GROUP) {
+        int groupId = SerialNumberManager.INSTANCE
+            .getGroupSerialNumber(aclEntry.getName());
+        aclEntryInt = NAME.BITS.combine(groupId, aclEntryInt);
+      }
+    }
+    return (int) aclEntryInt;
+  }
+
+  static AclEntry toAclEntry(int aclEntry) {
+    AclEntry.Builder builder = new AclEntry.Builder();
+    builder.setScope(getScope(aclEntry)).setType(getType(aclEntry))
+        .setPermission(getPermission(aclEntry));
+    if (getName(aclEntry) != null) {
+      builder.setName(getName(aclEntry));
+    }
+    return builder.build();
+  }
+
+  public static int[] toInt(List<AclEntry> aclEntries) {
+    int[] entries = new int[aclEntries.size()];
+    for (int i = 0; i < entries.length; i++) {
+      entries[i] = toInt(aclEntries.get(i));
+    }
+    return entries;
+  }
+
+  public static ImmutableList<AclEntry> toAclEntries(int[] entries) {
+    ImmutableList.Builder<AclEntry> b = new ImmutableList.Builder<AclEntry>();
+    for (int entry : entries) {
+      AclEntry aclEntry = toAclEntry(entry);
+      b.add(aclEntry);
+    }
+    return b.build();
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0653918d/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/AclFeature.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/AclFeature.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/AclFeature.java
index 1c5f469..e097b05 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/AclFeature.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/AclFeature.java
@@ -21,6 +21,7 @@ package org.apache.hadoop.hdfs.server.namenode;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.fs.permission.AclEntry;
 
+import com.google.common.base.Preconditions;
 import com.google.common.collect.ImmutableList;
 
 /**
@@ -31,13 +32,28 @@ public class AclFeature implements INode.Feature {
   public static final ImmutableList<AclEntry> EMPTY_ENTRY_LIST =
     ImmutableList.of();
 
-  private final ImmutableList<AclEntry> entries;
+  private final int [] entries;
 
-  public AclFeature(ImmutableList<AclEntry> entries) {
+  public AclFeature(int[] entries) {
     this.entries = entries;
   }
 
-  public ImmutableList<AclEntry> getEntries() {
-    return entries;
+  /**
+   * Get the number of entries present
+   */
+  int getEntriesSize() {
+    return entries.length;
+  }
+
+  /**
+   * Get the entry at the specified position
+   * @param pos Position of the entry to be obtained
+   * @return integer representation of AclEntry
+   * @throws IndexOutOfBoundsException if pos out of bound
+   */
+  int getEntryAt(int pos) {
+    Preconditions.checkPositionIndex(pos, entries.length,
+        "Invalid position for AclEntry");
+    return entries[pos];
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0653918d/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/AclStorage.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/AclStorage.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/AclStorage.java
index c15d64e..ac30597 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/AclStorage.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/AclStorage.java
@@ -20,6 +20,7 @@ package org.apache.hadoop.hdfs.server.namenode;
 import java.util.Collections;
 import java.util.List;
 
+import com.google.common.annotations.VisibleForTesting;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Lists;
 
@@ -76,7 +77,8 @@ final class AclStorage {
     }
 
     // Split parent's entries into access vs. default.
-    List<AclEntry> featureEntries = parent.getAclFeature().getEntries();
+    List<AclEntry> featureEntries = getEntriesFromAclFeature(parent
+        .getAclFeature());
     ScopedAclEntries scopedEntries = new ScopedAclEntries(featureEntries);
     List<AclEntry> parentDefaultEntries = scopedEntries.getDefaultEntries();
 
@@ -153,7 +155,25 @@ final class AclStorage {
    */
   public static List<AclEntry> readINodeAcl(INode inode, int snapshotId) {
     AclFeature f = inode.getAclFeature(snapshotId);
-    return f == null ? ImmutableList.<AclEntry> of() : f.getEntries();
+    return getEntriesFromAclFeature(f);
+  }
+
+  /**
+   * Build list of AclEntries from the AclFeature
+   * @param aclFeature AclFeature
+   * @return List of entries
+   */
+  @VisibleForTesting
+  static ImmutableList<AclEntry> getEntriesFromAclFeature(AclFeature aclFeature) {
+    if (aclFeature == null) {
+      return ImmutableList.<AclEntry> of();
+    }
+    ImmutableList.Builder<AclEntry> b = new ImmutableList.Builder<AclEntry>();
+    for (int pos = 0, entry; pos < aclFeature.getEntriesSize(); pos++) {
+      entry = aclFeature.getEntryAt(pos);
+      b.add(AclEntryStatusFormat.toAclEntry(entry));
+    }
+    return b.build();
   }
 
   /**
@@ -179,7 +199,7 @@ final class AclStorage {
 
     final List<AclEntry> existingAcl;
     // Split ACL entries stored in the feature into access vs. default.
-    List<AclEntry> featureEntries = f.getEntries();
+    List<AclEntry> featureEntries = getEntriesFromAclFeature(f);
     ScopedAclEntries scoped = new ScopedAclEntries(featureEntries);
     List<AclEntry> accessEntries = scoped.getAccessEntries();
     List<AclEntry> defaultEntries = scoped.getDefaultEntries();
@@ -235,7 +255,7 @@ final class AclStorage {
     }
 
     FsPermission perm = inode.getFsPermission();
-    List<AclEntry> featureEntries = f.getEntries();
+    List<AclEntry> featureEntries = getEntriesFromAclFeature(f);
     if (featureEntries.get(0).getScope() == AclEntryScope.ACCESS) {
       // Restore group permissions from the feature's entry to permission
       // bits, overwriting the mask, which is not part of a minimal ACL.
@@ -330,7 +350,7 @@ final class AclStorage {
 
     // Add all default entries to the feature.
     featureEntries.addAll(defaultEntries);
-    return new AclFeature(ImmutableList.copyOf(featureEntries));
+    return new AclFeature(AclEntryStatusFormat.toInt(featureEntries));
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0653918d/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormatPBINode.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormatPBINode.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormatPBINode.java
index 51f2606..26ca16a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormatPBINode.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormatPBINode.java
@@ -157,8 +157,9 @@ public final class FSImageFormatPBINode {
       }
 
       if (d.hasAcl()) {
-        dir.addAclFeature(new AclFeature(loadAclEntries(d.getAcl(),
-            state.getStringTable())));
+        int[] entries = AclEntryStatusFormat.toInt(loadAclEntries(
+            d.getAcl(), state.getStringTable()));
+        dir.addAclFeature(new AclFeature(entries));
       }
       if (d.hasXAttrs()) {
         dir.addXAttrFeature(new XAttrFeature(
@@ -294,8 +295,9 @@ public final class FSImageFormatPBINode {
           (byte)f.getStoragePolicyID());
 
       if (f.hasAcl()) {
-        file.addAclFeature(new AclFeature(loadAclEntries(f.getAcl(),
-            state.getStringTable())));
+        int[] entries = AclEntryStatusFormat.toInt(loadAclEntries(
+            f.getAcl(), state.getStringTable()));
+        file.addAclFeature(new AclFeature(entries));
       }
       
       if (f.hasXAttrs()) {
@@ -362,11 +364,13 @@ public final class FSImageFormatPBINode {
     private static AclFeatureProto.Builder buildAclEntries(AclFeature f,
         final SaverContext.DeduplicationMap<String> map) {
       AclFeatureProto.Builder b = AclFeatureProto.newBuilder();
-      for (AclEntry e : f.getEntries()) {
-        int v = ((map.getId(e.getName()) & ACL_ENTRY_NAME_MASK) << ACL_ENTRY_NAME_OFFSET)
-            | (e.getType().ordinal() << ACL_ENTRY_TYPE_OFFSET)
-            | (e.getScope().ordinal() << ACL_ENTRY_SCOPE_OFFSET)
-            | (e.getPermission().ordinal());
+      for (int pos = 0, e; pos < f.getEntriesSize(); pos++) {
+        e = f.getEntryAt(pos);
+        int nameId = map.getId(AclEntryStatusFormat.getName(e));
+        int v = ((nameId & ACL_ENTRY_NAME_MASK) << ACL_ENTRY_NAME_OFFSET)
+            | (AclEntryStatusFormat.getType(e).ordinal() << ACL_ENTRY_TYPE_OFFSET)
+            | (AclEntryStatusFormat.getScope(e).ordinal() << ACL_ENTRY_SCOPE_OFFSET)
+            | (AclEntryStatusFormat.getPermission(e).ordinal());
         b.addEntries(v);
       }
       return b;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0653918d/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSPermissionChecker.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSPermissionChecker.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSPermissionChecker.java
index f994f6b..54090f2 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSPermissionChecker.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSPermissionChecker.java
@@ -20,14 +20,12 @@ package org.apache.hadoop.hdfs.server.namenode;
 import java.util.Arrays;
 import java.util.Collections;
 import java.util.HashSet;
-import java.util.List;
 import java.util.Set;
 import java.util.Stack;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.fs.UnresolvedLinkException;
-import org.apache.hadoop.fs.permission.AclEntry;
 import org.apache.hadoop.fs.permission.AclEntryScope;
 import org.apache.hadoop.fs.permission.AclEntryType;
 import org.apache.hadoop.fs.permission.FsAction;
@@ -35,7 +33,6 @@ import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.util.ReadOnlyList;
 import org.apache.hadoop.security.AccessControlException;
 import org.apache.hadoop.security.UserGroupInformation;
-import org.apache.hadoop.util.StringUtils;
 
 /** 
  * Class that helps in checking file system permission.
@@ -50,12 +47,6 @@ class FSPermissionChecker {
   /** @return a string for throwing {@link AccessControlException} */
   private String toAccessControlString(INode inode, int snapshotId,
       FsAction access, FsPermission mode) {
-    return toAccessControlString(inode, snapshotId, access, mode, null);
-  }
-
-  /** @return a string for throwing {@link AccessControlException} */
-  private String toAccessControlString(INode inode, int snapshotId,
-      FsAction access, FsPermission mode, List<AclEntry> featureEntries) {
     StringBuilder sb = new StringBuilder("Permission denied: ")
       .append("user=").append(user).append(", ")
       .append("access=").append(access).append(", ")
@@ -64,9 +55,6 @@ class FSPermissionChecker {
       .append(inode.getGroupName(snapshotId)).append(':')
       .append(inode.isDirectory() ? 'd' : '-')
       .append(mode);
-    if (featureEntries != null) {
-      sb.append(':').append(StringUtils.join(",", featureEntries));
-    }
     return sb.toString();
   }
 
@@ -249,10 +237,10 @@ class FSPermissionChecker {
     FsPermission mode = inode.getFsPermission(snapshotId);
     AclFeature aclFeature = inode.getAclFeature(snapshotId);
     if (aclFeature != null) {
-      List<AclEntry> featureEntries = aclFeature.getEntries();
       // It's possible that the inode has a default ACL but no access ACL.
-      if (featureEntries.get(0).getScope() == AclEntryScope.ACCESS) {
-        checkAccessAcl(inode, snapshotId, access, mode, featureEntries);
+      int firstEntry = aclFeature.getEntryAt(0);
+      if (AclEntryStatusFormat.getScope(firstEntry) == AclEntryScope.ACCESS) {
+        checkAccessAcl(inode, snapshotId, access, mode, aclFeature);
         return;
       }
     }
@@ -294,11 +282,11 @@ class FSPermissionChecker {
    * @param snapshotId int snapshot ID
    * @param access FsAction requested permission
    * @param mode FsPermission mode from inode
-   * @param featureEntries List<AclEntry> ACL entries from AclFeature of inode
+   * @param aclFeature AclFeature of inode
    * @throws AccessControlException if the ACL denies permission
    */
   private void checkAccessAcl(INode inode, int snapshotId, FsAction access,
-      FsPermission mode, List<AclEntry> featureEntries)
+      FsPermission mode, AclFeature aclFeature)
       throws AccessControlException {
     boolean foundMatch = false;
 
@@ -312,17 +300,19 @@ class FSPermissionChecker {
 
     // Check named user and group entries if user was not denied by owner entry.
     if (!foundMatch) {
-      for (AclEntry entry: featureEntries) {
-        if (entry.getScope() == AclEntryScope.DEFAULT) {
+      for (int pos = 0, entry; pos < aclFeature.getEntriesSize(); pos++) {
+        entry = aclFeature.getEntryAt(pos);
+        if (AclEntryStatusFormat.getScope(entry) == AclEntryScope.DEFAULT) {
           break;
         }
-        AclEntryType type = entry.getType();
-        String name = entry.getName();
+        AclEntryType type = AclEntryStatusFormat.getType(entry);
+        String name = AclEntryStatusFormat.getName(entry);
         if (type == AclEntryType.USER) {
           // Use named user entry with mask from permission bits applied if user
           // matches name.
           if (user.equals(name)) {
-            FsAction masked = entry.getPermission().and(mode.getGroupAction());
+            FsAction masked = AclEntryStatusFormat.getPermission(entry).and(
+                mode.getGroupAction());
             if (masked.implies(access)) {
               return;
             }
@@ -336,7 +326,8 @@ class FSPermissionChecker {
           // it doesn't matter which is chosen, so exit early after first match.
           String group = name == null ? inode.getGroupName(snapshotId) : name;
           if (groups.contains(group)) {
-            FsAction masked = entry.getPermission().and(mode.getGroupAction());
+            FsAction masked = AclEntryStatusFormat.getPermission(entry).and(
+                mode.getGroupAction());
             if (masked.implies(access)) {
               return;
             }
@@ -352,7 +343,7 @@ class FSPermissionChecker {
     }
 
     throw new AccessControlException(
-      toAccessControlString(inode, snapshotId, access, mode, featureEntries));
+      toAccessControlString(inode, snapshotId, access, mode));
   }
 
   /** Guarded by {@link FSNamesystem#readLock()} */

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0653918d/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FSImageFormatPBSnapshot.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FSImageFormatPBSnapshot.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FSImageFormatPBSnapshot.java
index ff33225..86ba03d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FSImageFormatPBSnapshot.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FSImageFormatPBSnapshot.java
@@ -36,6 +36,7 @@ import java.util.Map;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.fs.permission.PermissionStatus;
+import org.apache.hadoop.hdfs.server.namenode.AclEntryStatusFormat;
 import org.apache.hadoop.hdfs.server.namenode.AclFeature;
 import org.apache.hadoop.hdfs.server.namenode.FSDirectory;
 import org.apache.hadoop.hdfs.server.namenode.FSImageFormatPBINode;
@@ -208,8 +209,10 @@ public class FSImageFormatPBSnapshot {
 
           AclFeature acl = null;
           if (fileInPb.hasAcl()) {
-            acl = new AclFeature(FSImageFormatPBINode.Loader.loadAclEntries(
-                fileInPb.getAcl(), state.getStringTable()));
+            int[] entries = AclEntryStatusFormat
+                .toInt(FSImageFormatPBINode.Loader.loadAclEntries(
+                    fileInPb.getAcl(), state.getStringTable()));
+            acl = new AclFeature(entries);
           }
           XAttrFeature xAttrs = null;
           if (fileInPb.hasXAttrs()) {
@@ -309,8 +312,10 @@ public class FSImageFormatPBSnapshot {
               dirCopyInPb.getPermission(), state.getStringTable());
           AclFeature acl = null;
           if (dirCopyInPb.hasAcl()) {
-            acl = new AclFeature(FSImageFormatPBINode.Loader.loadAclEntries(
-                dirCopyInPb.getAcl(), state.getStringTable()));
+            int[] entries = AclEntryStatusFormat
+                .toInt(FSImageFormatPBINode.Loader.loadAclEntries(
+                    dirCopyInPb.getAcl(), state.getStringTable()));
+            acl = new AclFeature(entries);
           }
           XAttrFeature xAttrs = null;
           if (dirCopyInPb.hasXAttrs()) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0653918d/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/FSAclBaseTest.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/FSAclBaseTest.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/FSAclBaseTest.java
index 5066feb..aff133f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/FSAclBaseTest.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/FSAclBaseTest.java
@@ -1395,8 +1395,8 @@ public abstract class FSAclBaseTest {
       // Intentionally capturing a reference to the entries, not using nested
       // calls.  This way, we get compile-time enforcement that the entries are
       // stored in an ImmutableList.
-      ImmutableList<AclEntry> entries = aclFeature.getEntries();
-      assertNotNull(entries);
+      ImmutableList<AclEntry> entries = AclStorage
+          .getEntriesFromAclFeature(aclFeature);
       assertFalse(entries.isEmpty());
     } else {
       assertNull(aclFeature);


[11/29] hadoop git commit: HADOOP-11348. Remove unused variable from CMake error message for finding openssl (Dian Fu via Colin P. McCabe)

Posted by vi...@apache.org.
HADOOP-11348. Remove unused variable from CMake error message for finding openssl (Dian Fu via Colin P. McCabe)


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

Branch: refs/heads/HDFS-EC
Commit: 565b0e60a8fc4ae5bc0083cc6a6ddb2d01952f32
Parents: 1bbcc3d
Author: Colin Patrick Mccabe <cm...@cloudera.com>
Authored: Thu Dec 4 12:51:42 2014 -0800
Committer: Colin Patrick Mccabe <cm...@cloudera.com>
Committed: Thu Dec 4 12:52:39 2014 -0800

----------------------------------------------------------------------
 hadoop-common-project/hadoop-common/CHANGES.txt        | 3 +++
 hadoop-common-project/hadoop-common/src/CMakeLists.txt | 2 +-
 2 files changed, 4 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/565b0e60/hadoop-common-project/hadoop-common/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/CHANGES.txt b/hadoop-common-project/hadoop-common/CHANGES.txt
index f53bceb..f2a086e 100644
--- a/hadoop-common-project/hadoop-common/CHANGES.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES.txt
@@ -499,6 +499,9 @@ Release 2.7.0 - UNRELEASED
     HADOOP-11332. KerberosAuthenticator#doSpnegoSequence should check if
     kerberos TGT is available in the subject. (Dian Fu via atm)
 
+    HADOOP-11348. Remove unused variable from CMake error message for finding
+    openssl (Dian Fu via Colin P. McCabe)
+
 Release 2.6.0 - 2014-11-18
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/565b0e60/hadoop-common-project/hadoop-common/src/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/CMakeLists.txt b/hadoop-common-project/hadoop-common/src/CMakeLists.txt
index b8ac460..29fa2b8 100644
--- a/hadoop-common-project/hadoop-common/src/CMakeLists.txt
+++ b/hadoop-common-project/hadoop-common/src/CMakeLists.txt
@@ -202,7 +202,7 @@ if (USABLE_OPENSSL)
         "${D}/crypto/OpensslCipher.c"
         "${D}/crypto/random/OpensslSecureRandom.c")
 else (USABLE_OPENSSL)
-    MESSAGE("Cannot find a usable OpenSSL library.  OPENSSL_LIBRARY=${OPENSSL_LIBRARY}, OPENSSL_INCLUDE_DIR=${OPENSSL_INCLUDE_DIR}, CUSTOM_OPENSSL_INCLUDE_DIR=${CUSTOM_OPENSSL_INCLUDE_DIR}, CUSTOM_OPENSSL_PREFIX=${CUSTOM_OPENSSL_PREFIX}, CUSTOM_OPENSSL_INCLUDE=${CUSTOM_OPENSSL_INCLUDE}")
+    MESSAGE("Cannot find a usable OpenSSL library.  OPENSSL_LIBRARY=${OPENSSL_LIBRARY}, OPENSSL_INCLUDE_DIR=${OPENSSL_INCLUDE_DIR}, CUSTOM_OPENSSL_LIB=${CUSTOM_OPENSSL_LIB}, CUSTOM_OPENSSL_PREFIX=${CUSTOM_OPENSSL_PREFIX}, CUSTOM_OPENSSL_INCLUDE=${CUSTOM_OPENSSL_INCLUDE}")
     IF(REQUIRE_OPENSSL)
         MESSAGE(FATAL_ERROR "Terminating build because require.openssl was specified.")
     ENDIF(REQUIRE_OPENSSL)


[17/29] hadoop git commit: HDFS-7478. Move org.apache.hadoop.hdfs.server.namenode.NNConf to FSNamesystem. Contributed by Li Lu.

Posted by vi...@apache.org.
HDFS-7478. Move org.apache.hadoop.hdfs.server.namenode.NNConf to FSNamesystem. Contributed by Li Lu.


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

Branch: refs/heads/HDFS-EC
Commit: 6a5596e3b4443462fc86f800b3c2eb839d44c3bd
Parents: 2829b7a
Author: Haohui Mai <wh...@apache.org>
Authored: Fri Dec 5 10:55:13 2014 -0800
Committer: Haohui Mai <wh...@apache.org>
Committed: Fri Dec 5 10:55:13 2014 -0800

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |   3 +
 .../hdfs/server/namenode/FSNamesystem.java      |  66 +++++++++---
 .../hadoop/hdfs/server/namenode/NNConf.java     | 104 -------------------
 3 files changed, 54 insertions(+), 119 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/6a5596e3/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index 02f41cc..c6cb185 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -430,6 +430,9 @@ Release 2.7.0 - UNRELEASED
     HDFS-7468. Moving verify* functions to corresponding classes.
     (Li Lu via wheat9)
 
+    HDFS-7478. Move org.apache.hadoop.hdfs.server.namenode.NNConf to
+    FSNamesystem. (Li Lu via wheat9)
+
   OPTIMIZATIONS
 
     HDFS-7454. Reduce memory footprint for AclEntries in NameNode.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6a5596e3/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 a6e88c6..22039fc 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
@@ -532,7 +532,9 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
 
   private final RetryCache retryCache;
 
-  private final NNConf nnConf;
+  private final boolean aclsEnabled;
+  private final boolean xattrsEnabled;
+  private final int xattrMaxSize;
 
   private KeyProviderCryptoExtension provider = null;
 
@@ -848,7 +850,23 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
       this.isDefaultAuditLogger = auditLoggers.size() == 1 &&
         auditLoggers.get(0) instanceof DefaultAuditLogger;
       this.retryCache = ignoreRetryCache ? null : initRetryCache(conf);
-      this.nnConf = new NNConf(conf);
+
+      this.aclsEnabled = conf.getBoolean(
+          DFSConfigKeys.DFS_NAMENODE_ACLS_ENABLED_KEY,
+          DFSConfigKeys.DFS_NAMENODE_ACLS_ENABLED_DEFAULT);
+      LOG.info("ACLs enabled? " + aclsEnabled);
+      this.xattrsEnabled = conf.getBoolean(
+          DFSConfigKeys.DFS_NAMENODE_XATTRS_ENABLED_KEY,
+          DFSConfigKeys.DFS_NAMENODE_XATTRS_ENABLED_DEFAULT);
+      LOG.info("XAttrs enabled? " + xattrsEnabled);
+      this.xattrMaxSize = conf.getInt(
+          DFSConfigKeys.DFS_NAMENODE_MAX_XATTR_SIZE_KEY,
+          DFSConfigKeys.DFS_NAMENODE_MAX_XATTR_SIZE_DEFAULT);
+      Preconditions.checkArgument(xattrMaxSize >= 0,
+          "Cannot set a negative value for the maximum size of an xattr (%s).",
+          DFSConfigKeys.DFS_NAMENODE_MAX_XATTR_SIZE_KEY);
+      final String unlimited = xattrMaxSize == 0 ? " (unlimited)" : "";
+      LOG.info("Maximum size of an xattr: " + xattrMaxSize + unlimited);
     } catch(IOException e) {
       LOG.error(getClass().getSimpleName() + " initialization failed.", e);
       close();
@@ -7827,7 +7845,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
   void modifyAclEntries(final String srcArg, List<AclEntry> aclSpec)
       throws IOException {
     String src = srcArg;
-    nnConf.checkAclsConfigFlag();
+    checkAclsConfigFlag();
     HdfsFileStatus resultingStat = null;
     FSPermissionChecker pc = getPermissionChecker();
     checkOperation(OperationCategory.WRITE);
@@ -7854,7 +7872,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
   void removeAclEntries(final String srcArg, List<AclEntry> aclSpec)
       throws IOException {
     String src = srcArg;
-    nnConf.checkAclsConfigFlag();
+    checkAclsConfigFlag();
     HdfsFileStatus resultingStat = null;
     FSPermissionChecker pc = getPermissionChecker();
     checkOperation(OperationCategory.WRITE);
@@ -7880,7 +7898,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
 
   void removeDefaultAcl(final String srcArg) throws IOException {
     String src = srcArg;
-    nnConf.checkAclsConfigFlag();
+    checkAclsConfigFlag();
     HdfsFileStatus resultingStat = null;
     FSPermissionChecker pc = getPermissionChecker();
     checkOperation(OperationCategory.WRITE);
@@ -7906,7 +7924,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
 
   void removeAcl(final String srcArg) throws IOException {
     String src = srcArg;
-    nnConf.checkAclsConfigFlag();
+    checkAclsConfigFlag();
     HdfsFileStatus resultingStat = null;
     FSPermissionChecker pc = getPermissionChecker();
     checkOperation(OperationCategory.WRITE);
@@ -7932,7 +7950,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
 
   void setAcl(final String srcArg, List<AclEntry> aclSpec) throws IOException {
     String src = srcArg;
-    nnConf.checkAclsConfigFlag();
+    checkAclsConfigFlag();
     HdfsFileStatus resultingStat = null;
     FSPermissionChecker pc = getPermissionChecker();
     checkOperation(OperationCategory.WRITE);
@@ -7957,7 +7975,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
   }
 
   AclStatus getAclStatus(String src) throws IOException {
-    nnConf.checkAclsConfigFlag();
+    checkAclsConfigFlag();
     FSPermissionChecker pc = getPermissionChecker();
     checkOperation(OperationCategory.READ);
     byte[][] pathComponents = FSDirectory.getPathComponentsForReservedPath(src);
@@ -8141,7 +8159,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
   private void setXAttrInt(final String srcArg, XAttr xAttr,
       EnumSet<XAttrSetFlag> flag, boolean logRetryCache) throws IOException {
     String src = srcArg;
-    nnConf.checkXAttrsConfigFlag();
+    checkXAttrsConfigFlag();
     checkXAttrSize(xAttr);
     HdfsFileStatus resultingStat = null;
     FSPermissionChecker pc = getPermissionChecker();
@@ -8172,17 +8190,17 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
    * the configured limit. Setting a limit of zero disables this check.
    */
   private void checkXAttrSize(XAttr xAttr) {
-    if (nnConf.xattrMaxSize == 0) {
+    if (xattrMaxSize == 0) {
       return;
     }
     int size = xAttr.getName().getBytes(Charsets.UTF_8).length;
     if (xAttr.getValue() != null) {
       size += xAttr.getValue().length;
     }
-    if (size > nnConf.xattrMaxSize) {
+    if (size > xattrMaxSize) {
       throw new HadoopIllegalArgumentException(
           "The XAttr is too big. The maximum combined size of the"
-          + " name and value is " + nnConf.xattrMaxSize
+          + " name and value is " + xattrMaxSize
           + ", but the total size is " + size);
     }
   }
@@ -8190,7 +8208,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
   List<XAttr> getXAttrs(final String srcArg, List<XAttr> xAttrs)
       throws IOException {
     String src = srcArg;
-    nnConf.checkXAttrsConfigFlag();
+    checkXAttrsConfigFlag();
     FSPermissionChecker pc = getPermissionChecker();
     final boolean isRawPath = FSDirectory.isReservedRawName(src);
     boolean getAll = xAttrs == null || xAttrs.isEmpty();
@@ -8247,7 +8265,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
   }
 
   List<XAttr> listXAttrs(String src) throws IOException {
-    nnConf.checkXAttrsConfigFlag();
+    checkXAttrsConfigFlag();
     final FSPermissionChecker pc = getPermissionChecker();
     final boolean isRawPath = FSDirectory.isReservedRawName(src);
     checkOperation(OperationCategory.READ);
@@ -8297,7 +8315,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
   void removeXAttrInt(final String srcArg, XAttr xAttr, boolean logRetryCache)
       throws IOException {
     String src = srcArg;
-    nnConf.checkXAttrsConfigFlag();
+    checkXAttrsConfigFlag();
     HdfsFileStatus resultingStat = null;
     FSPermissionChecker pc = getPermissionChecker();
     XAttrPermissionFilter.checkPermissionForApi(pc, xAttr,
@@ -8452,5 +8470,23 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
       logger.addAppender(asyncAppender);        
     }
   }
+
+  private void checkAclsConfigFlag() throws AclException {
+    if (!aclsEnabled) {
+      throw new AclException(String.format(
+          "The ACL operation has been rejected.  "
+              + "Support for ACLs has been disabled by setting %s to false.",
+          DFSConfigKeys.DFS_NAMENODE_ACLS_ENABLED_KEY));
+    }
+  }
+
+  private void checkXAttrsConfigFlag() throws IOException {
+    if (!xattrsEnabled) {
+      throw new IOException(String.format(
+          "The XAttr operation has been rejected.  "
+              + "Support for XAttrs has been disabled by setting %s to false.",
+          DFSConfigKeys.DFS_NAMENODE_XATTRS_ENABLED_KEY));
+    }
+  }
 }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6a5596e3/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NNConf.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NNConf.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NNConf.java
deleted file mode 100644
index 94dc6b9..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NNConf.java
+++ /dev/null
@@ -1,104 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hdfs.server.namenode;
-
-import java.io.IOException;
-
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hdfs.DFSConfigKeys;
-import org.apache.hadoop.hdfs.protocol.AclException;
-
-import com.google.common.base.Preconditions;
-
-/**
- * This class is a common place for NN configuration.
- */
-@InterfaceAudience.Private
-final class NNConf {
-  /**
-   * Support for ACLs is controlled by a configuration flag. If the 
-   * configuration flag is false, then the NameNode will reject all 
-   * ACL-related operations.
-   */
-  private final boolean aclsEnabled;
-  
-  /**
-   * Support for XAttrs is controlled by a configuration flag. If the 
-   * configuration flag is false, then the NameNode will reject all 
-   * XAttr-related operations.
-   */
-  private final boolean xattrsEnabled;
-  
-  /**
-   * Maximum size of a single name-value extended attribute.
-   */
-  final int xattrMaxSize;
-
-  /**
-   * Creates a new NNConf from configuration.
-   *
-   * @param conf Configuration to check
-   */
-  public NNConf(Configuration conf) {
-    aclsEnabled = conf.getBoolean(DFSConfigKeys.DFS_NAMENODE_ACLS_ENABLED_KEY,
-      DFSConfigKeys.DFS_NAMENODE_ACLS_ENABLED_DEFAULT);
-    LogFactory.getLog(NNConf.class).info("ACLs enabled? " + aclsEnabled);
-    xattrsEnabled = conf.getBoolean(
-        DFSConfigKeys.DFS_NAMENODE_XATTRS_ENABLED_KEY,
-        DFSConfigKeys.DFS_NAMENODE_XATTRS_ENABLED_DEFAULT);
-    LogFactory.getLog(NNConf.class).info("XAttrs enabled? " + xattrsEnabled);
-    xattrMaxSize = conf.getInt(
-        DFSConfigKeys.DFS_NAMENODE_MAX_XATTR_SIZE_KEY,
-        DFSConfigKeys.DFS_NAMENODE_MAX_XATTR_SIZE_DEFAULT);
-    Preconditions.checkArgument(xattrMaxSize >= 0,
-        "Cannot set a negative value for the maximum size of an xattr (%s).",
-        DFSConfigKeys.DFS_NAMENODE_MAX_XATTR_SIZE_KEY);
-    final String unlimited = xattrMaxSize == 0 ? " (unlimited)" : "";
-    LogFactory.getLog(NNConf.class).info(
-        "Maximum size of an xattr: " + xattrMaxSize + unlimited);
-  }
-
-  /**
-   * Checks the flag on behalf of an ACL API call.
-   *
-   * @throws AclException if ACLs are disabled
-   */
-  public void checkAclsConfigFlag() throws AclException {
-    if (!aclsEnabled) {
-      throw new AclException(String.format(
-        "The ACL operation has been rejected.  "
-        + "Support for ACLs has been disabled by setting %s to false.",
-        DFSConfigKeys.DFS_NAMENODE_ACLS_ENABLED_KEY));
-    }
-  }
-  
-  /**
-   * Checks the flag on behalf of an XAttr API call.
-   * @throws IOException if XAttrs are disabled
-   */
-  public void checkXAttrsConfigFlag() throws IOException {
-    if (!xattrsEnabled) {
-      throw new IOException(String.format(
-        "The XAttr operation has been rejected.  "
-        + "Support for XAttrs has been disabled by setting %s to false.",
-        DFSConfigKeys.DFS_NAMENODE_XATTRS_ENABLED_KEY));
-    }
-  }
-}


[22/29] hadoop git commit: HDFS-7474. Avoid resolving path in FSPermissionChecker. Contributed by Jing Zhao.

Posted by vi...@apache.org.
HDFS-7474. Avoid resolving path in FSPermissionChecker. Contributed by Jing Zhao.


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

Branch: refs/heads/HDFS-EC
Commit: 475c6b4978045d55d1ebcea69cc9a2f24355aca2
Parents: 4b13082
Author: Jing Zhao <ji...@apache.org>
Authored: Fri Dec 5 14:17:17 2014 -0800
Committer: Jing Zhao <ji...@apache.org>
Committed: Fri Dec 5 14:17:17 2014 -0800

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |   2 +
 .../server/namenode/EncryptionZoneManager.java  |   4 +-
 .../hdfs/server/namenode/FSDirConcatOp.java     |   9 +-
 .../hdfs/server/namenode/FSDirMkdirOp.java      |  17 +-
 .../hdfs/server/namenode/FSDirRenameOp.java     |  20 +-
 .../hdfs/server/namenode/FSDirSnapshotOp.java   |  48 ++--
 .../server/namenode/FSDirStatAndListingOp.java  |  35 +--
 .../hdfs/server/namenode/FSDirectory.java       |  99 +++----
 .../hdfs/server/namenode/FSEditLogLoader.java   |  16 +-
 .../hdfs/server/namenode/FSNamesystem.java      | 270 ++++++++-----------
 .../server/namenode/FSPermissionChecker.java    |  22 +-
 .../hdfs/server/namenode/INodesInPath.java      |  21 +-
 .../namenode/snapshot/SnapshotManager.java      |  50 ++--
 .../namenode/TestFSPermissionChecker.java       |  10 +-
 .../server/namenode/TestSnapshotPathINodes.java |  20 +-
 .../namenode/snapshot/TestSnapshotManager.java  |  14 +-
 16 files changed, 295 insertions(+), 362 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/475c6b49/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index 22f462f..d4db732 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -433,6 +433,8 @@ Release 2.7.0 - UNRELEASED
     HDFS-7478. Move org.apache.hadoop.hdfs.server.namenode.NNConf to
     FSNamesystem. (Li Lu via wheat9)
 
+    HDFS-7474. Avoid resolving path in FSPermissionChecker. (jing9)
+
   OPTIMIZATIONS
 
     HDFS-7454. Reduce memory footprint for AclEntries in NameNode.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/475c6b49/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 0d7ced9..135979f 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
@@ -285,12 +285,12 @@ public class EncryptionZoneManager {
       CryptoProtocolVersion version, String keyName)
       throws IOException {
     assert dir.hasWriteLock();
-    if (dir.isNonEmptyDirectory(src)) {
+    final INodesInPath srcIIP = dir.getINodesInPath4Write(src, false);
+    if (dir.isNonEmptyDirectory(srcIIP)) {
       throw new IOException(
           "Attempt to create an encryption zone for a non-empty directory.");
     }
 
-    final INodesInPath srcIIP = dir.getINodesInPath4Write(src, false);
     if (srcIIP != null &&
         srcIIP.getLastINode() != null &&
         !srcIIP.getLastINode().isDirectory()) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/475c6b49/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirConcatOp.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirConcatOp.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirConcatOp.java
index 12feb33..c2e0f08 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirConcatOp.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirConcatOp.java
@@ -53,15 +53,17 @@ class FSDirConcatOp {
       }
     }
 
+    final INodesInPath trgIip = fsd.getINodesInPath4Write(target);
     // write permission for the target
     if (fsd.isPermissionEnabled()) {
       FSPermissionChecker pc = fsd.getPermissionChecker();
-      fsd.checkPathAccess(pc, target, FsAction.WRITE);
+      fsd.checkPathAccess(pc, trgIip, FsAction.WRITE);
 
       // and srcs
       for(String aSrc: srcs) {
-        fsd.checkPathAccess(pc, aSrc, FsAction.READ); // read the file
-        fsd.checkParentAccess(pc, aSrc, FsAction.WRITE); // for delete
+        final INodesInPath srcIip = fsd.getINodesInPath4Write(aSrc);
+        fsd.checkPathAccess(pc, srcIip, FsAction.READ); // read the file
+        fsd.checkParentAccess(pc, srcIip, FsAction.WRITE); // for delete
       }
     }
 
@@ -72,7 +74,6 @@ class FSDirConcatOp {
     // replication and blocks sizes should be the same for ALL the blocks
 
     // check the target
-    final INodesInPath trgIip = fsd.getINodesInPath4Write(target);
     if (fsd.getEZForPath(trgIip) != null) {
       throw new HadoopIllegalArgumentException(
           "concat can not be called for files in an encryption zone.");

http://git-wip-us.apache.org/repos/asf/hadoop/blob/475c6b49/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirMkdirOp.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirMkdirOp.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirMkdirOp.java
index 01cb57f..af9e925 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirMkdirOp.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirMkdirOp.java
@@ -53,17 +53,18 @@ class FSDirMkdirOp {
     byte[][] pathComponents = FSDirectory.getPathComponentsForReservedPath
         (src);
     src = fsd.resolvePath(pc, src, pathComponents);
+    INodesInPath iip = fsd.getINodesInPath4Write(src);
     if (fsd.isPermissionEnabled()) {
-      fsd.checkTraverse(pc, src);
+      fsd.checkTraverse(pc, iip);
     }
 
-    if (!isDirMutable(fsd, src)) {
+    if (!isDirMutable(fsd, iip)) {
       if (fsd.isPermissionEnabled()) {
-        fsd.checkAncestorAccess(pc, src, FsAction.WRITE);
+        fsd.checkAncestorAccess(pc, iip, FsAction.WRITE);
       }
 
       if (!createParent) {
-        fsd.verifyParentDir(src);
+        fsd.verifyParentDir(iip, src);
       }
 
       // validate that we have enough inodes. This is, at best, a
@@ -203,13 +204,11 @@ class FSDirMkdirOp {
    * Check whether the path specifies a directory
    * @throws SnapshotAccessControlException if path is in RO snapshot
    */
-  private static boolean isDirMutable(
-      FSDirectory fsd, String src) throws UnresolvedLinkException,
-      SnapshotAccessControlException {
-    src = FSDirectory.normalizePath(src);
+  private static boolean isDirMutable(FSDirectory fsd, INodesInPath iip)
+      throws SnapshotAccessControlException {
     fsd.readLock();
     try {
-      INode node = fsd.getINode4Write(src, false);
+      INode node = iip.getLastINode();
       return node != null && node.isDirectory();
     } finally {
       fsd.readUnlock();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/475c6b49/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirRenameOp.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirRenameOp.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirRenameOp.java
index 08241c4..9f3983a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirRenameOp.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirRenameOp.java
@@ -492,11 +492,13 @@ class FSDirRenameOp {
       // Rename does not operates on link targets
       // Do not resolveLink when checking permissions of src and dst
       // Check write access to parent of src
-      fsd.checkPermission(pc, src, false, null, FsAction.WRITE, null, null,
-          false, false);
+      INodesInPath srcIIP = fsd.getINodesInPath(src, false);
+      fsd.checkPermission(pc, srcIIP, false, null, FsAction.WRITE, null, null,
+          false);
+      INodesInPath dstIIP = fsd.getINodesInPath(actualdst, false);
       // Check write access to ancestor of dst
-      fsd.checkPermission(pc, actualdst, false, FsAction.WRITE, null, null,
-          null, false, false);
+      fsd.checkPermission(pc, dstIIP, false, FsAction.WRITE, null, null,
+          null, false);
     }
 
     long mtime = now();
@@ -518,11 +520,13 @@ class FSDirRenameOp {
       // Rename does not operates on link targets
       // Do not resolveLink when checking permissions of src and dst
       // Check write access to parent of src
-      fsd.checkPermission(pc, src, false, null, FsAction.WRITE, null, null,
-          false, false);
+      INodesInPath srcIIP = fsd.getINodesInPath(src, false);
+      fsd.checkPermission(pc, srcIIP, false, null, FsAction.WRITE, null, null,
+          false);
       // Check write access to ancestor of dst
-      fsd.checkPermission(pc, dst, false, FsAction.WRITE, null, null, null,
-          false, false);
+      INodesInPath dstIIP = fsd.getINodesInPath(dst, false);
+      fsd.checkPermission(pc, dstIIP, false, FsAction.WRITE, null, null, null,
+          false);
     }
 
     long mtime = now();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/475c6b49/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirSnapshotOp.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirSnapshotOp.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirSnapshotOp.java
index bfd7019..f295e06 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirSnapshotOp.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirSnapshotOp.java
@@ -81,27 +81,24 @@ class FSDirSnapshotOp {
       FSDirectory fsd, SnapshotManager snapshotManager, String snapshotRoot,
       String snapshotName, boolean logRetryCache)
       throws IOException {
-    final FSPermissionChecker pc = fsd.getPermissionChecker();
-
-    String snapshotPath = null;
+    final INodesInPath iip = fsd.getINodesInPath4Write(snapshotRoot);
     if (fsd.isPermissionEnabled()) {
-      fsd.checkOwner(pc, snapshotRoot);
+      FSPermissionChecker pc = fsd.getPermissionChecker();
+      fsd.checkOwner(pc, iip);
     }
 
     if (snapshotName == null || snapshotName.isEmpty()) {
       snapshotName = Snapshot.generateDefaultSnapshotName();
+    } else if (!DFSUtil.isValidNameForComponent(snapshotName)) {
+      throw new InvalidPathException("Invalid snapshot name: " + snapshotName);
     }
 
-    if(snapshotName != null){
-      if (!DFSUtil.isValidNameForComponent(snapshotName)) {
-        throw new InvalidPathException("Invalid snapshot name: " +
-            snapshotName);
-      }
-    }
+    String snapshotPath = null;
     verifySnapshotName(fsd, snapshotName, snapshotRoot);
     fsd.writeLock();
     try {
-      snapshotPath = snapshotManager.createSnapshot(snapshotRoot, snapshotName);
+      snapshotPath = snapshotManager.createSnapshot(iip, snapshotRoot,
+          snapshotName);
     } finally {
       fsd.writeUnlock();
     }
@@ -114,15 +111,16 @@ class FSDirSnapshotOp {
   static void renameSnapshot(FSDirectory fsd, SnapshotManager snapshotManager,
       String path, String snapshotOldName, String snapshotNewName,
       boolean logRetryCache) throws IOException {
-
+    final INodesInPath iip = fsd.getINodesInPath4Write(path);
     if (fsd.isPermissionEnabled()) {
       FSPermissionChecker pc = fsd.getPermissionChecker();
-        fsd.checkOwner(pc, path);
+      fsd.checkOwner(pc, iip);
     }
     verifySnapshotName(fsd, snapshotNewName, path);
     fsd.writeLock();
     try {
-      snapshotManager.renameSnapshot(path, snapshotOldName, snapshotNewName);
+      snapshotManager.renameSnapshot(iip, path, snapshotOldName,
+          snapshotNewName);
     } finally {
       fsd.writeUnlock();
     }
@@ -142,8 +140,8 @@ class FSDirSnapshotOp {
     }
   }
 
-  static SnapshotDiffReport getSnapshotDiffReport(
-      FSDirectory fsd, SnapshotManager snapshotManager, String path,
+  static SnapshotDiffReport getSnapshotDiffReport(FSDirectory fsd,
+      SnapshotManager snapshotManager, String path,
       String fromSnapshot, String toSnapshot) throws IOException {
     SnapshotDiffReport diffs;
     final FSPermissionChecker pc = fsd.getPermissionChecker();
@@ -153,7 +151,8 @@ class FSDirSnapshotOp {
         checkSubtreeReadPermission(fsd, pc, path, fromSnapshot);
         checkSubtreeReadPermission(fsd, pc, path, toSnapshot);
       }
-      diffs = snapshotManager.diff(path, fromSnapshot, toSnapshot);
+      INodesInPath iip = fsd.getINodesInPath(path, true);
+      diffs = snapshotManager.diff(iip, path, fromSnapshot, toSnapshot);
     } finally {
       fsd.readUnlock();
     }
@@ -170,18 +169,18 @@ class FSDirSnapshotOp {
       FSDirectory fsd, SnapshotManager snapshotManager, String snapshotRoot,
       String snapshotName, boolean logRetryCache)
       throws IOException {
-    final FSPermissionChecker pc = fsd.getPermissionChecker();
-
-    INode.BlocksMapUpdateInfo collectedBlocks = new INode.BlocksMapUpdateInfo();
+    final INodesInPath iip = fsd.getINodesInPath4Write(snapshotRoot);
     if (fsd.isPermissionEnabled()) {
-      fsd.checkOwner(pc, snapshotRoot);
+      FSPermissionChecker pc = fsd.getPermissionChecker();
+      fsd.checkOwner(pc, iip);
     }
 
+    INode.BlocksMapUpdateInfo collectedBlocks = new INode.BlocksMapUpdateInfo();
     ChunkedArrayList<INode> removedINodes = new ChunkedArrayList<INode>();
     fsd.writeLock();
     try {
-      snapshotManager.deleteSnapshot(snapshotRoot, snapshotName,
-          collectedBlocks, removedINodes);
+      snapshotManager.deleteSnapshot(iip, snapshotName, collectedBlocks,
+          removedINodes);
       fsd.removeFromInodeMap(removedINodes);
     } finally {
       fsd.writeUnlock();
@@ -199,7 +198,8 @@ class FSDirSnapshotOp {
     final String fromPath = snapshot == null ?
         snapshottablePath : Snapshot.getSnapshotPath(snapshottablePath,
         snapshot);
-    fsd.checkPermission(pc, fromPath, false, null, null, FsAction.READ,
+    INodesInPath iip = fsd.getINodesInPath(fromPath, true);
+    fsd.checkPermission(pc, iip, false, null, null, FsAction.READ,
         FsAction.READ);
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/475c6b49/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirStatAndListingOp.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirStatAndListingOp.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirStatAndListingOp.java
index 35b3a6b..a8c3c16 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirStatAndListingOp.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirStatAndListingOp.java
@@ -45,15 +45,14 @@ import java.io.IOException;
 import java.util.Arrays;
 
 class FSDirStatAndListingOp {
-  static DirectoryListing getListingInt(
-      FSDirectory fsd, final String srcArg, byte[] startAfter,
-      boolean needLocation)
-    throws IOException {
-    String src = srcArg;
+  static DirectoryListing getListingInt(FSDirectory fsd, final String srcArg,
+      byte[] startAfter, boolean needLocation) throws IOException {
     FSPermissionChecker pc = fsd.getPermissionChecker();
-    byte[][] pathComponents = FSDirectory.getPathComponentsForReservedPath(src);
-    String startAfterString = new String(startAfter);
-    src = fsd.resolvePath(pc, src, pathComponents);
+    byte[][] pathComponents = FSDirectory
+        .getPathComponentsForReservedPath(srcArg);
+    final String startAfterString = new String(startAfter);
+    final String src = fsd.resolvePath(pc, srcArg, pathComponents);
+    final INodesInPath iip = fsd.getINodesInPath(src, true);
 
     // Get file name when startAfter is an INodePath
     if (FSDirectory.isReservedName(startAfterString)) {
@@ -73,9 +72,9 @@ class FSDirStatAndListingOp {
     boolean isSuperUser = true;
     if (fsd.isPermissionEnabled()) {
       if (fsd.isDir(src)) {
-        fsd.checkPathAccess(pc, src, FsAction.READ_EXECUTE);
+        fsd.checkPathAccess(pc, iip, FsAction.READ_EXECUTE);
       } else {
-        fsd.checkTraverse(pc, src);
+        fsd.checkTraverse(pc, iip);
       }
       isSuperUser = pc.isSuperUser();
     }
@@ -102,10 +101,10 @@ class FSDirStatAndListingOp {
     FSPermissionChecker pc = fsd.getPermissionChecker();
     byte[][] pathComponents = FSDirectory.getPathComponentsForReservedPath(src);
     src = fsd.resolvePath(pc, src, pathComponents);
+    final INodesInPath iip = fsd.getINodesInPath(src, resolveLink);
     boolean isSuperUser = true;
     if (fsd.isPermissionEnabled()) {
-      fsd.checkPermission(pc, src, false, null, null, null, null, false,
-          resolveLink);
+      fsd.checkPermission(pc, iip, false, null, null, null, null, false);
       isSuperUser = pc.isSuperUser();
     }
     return getFileInfo(fsd, src, resolveLink,
@@ -119,10 +118,13 @@ class FSDirStatAndListingOp {
     FSPermissionChecker pc = fsd.getPermissionChecker();
     byte[][] pathComponents = FSDirectory.getPathComponentsForReservedPath(src);
     src = fsd.resolvePath(pc, src, pathComponents);
+    final INodesInPath iip = fsd.getINodesInPath(src, true);
     if (fsd.isPermissionEnabled()) {
-      fsd.checkTraverse(pc, src);
+      fsd.checkTraverse(pc, iip);
     }
-    return !INodeFile.valueOf(fsd.getINode(src), src).isUnderConstruction();
+    INode[] inodes = iip.getINodes();
+    return !INodeFile.valueOf(inodes[inodes.length - 1],
+        src).isUnderConstruction();
   }
 
   static ContentSummary getContentSummary(
@@ -130,8 +132,9 @@ class FSDirStatAndListingOp {
     byte[][] pathComponents = FSDirectory.getPathComponentsForReservedPath(src);
     FSPermissionChecker pc = fsd.getPermissionChecker();
     src = fsd.resolvePath(pc, src, pathComponents);
+    final INodesInPath iip = fsd.getINodesInPath(src, true);
     if (fsd.isPermissionEnabled()) {
-      fsd.checkPermission(pc, src, false, null, null, null,
+      fsd.checkPermission(pc, iip, false, null, null, null,
           FsAction.READ_EXECUTE);
     }
     return getContentSummaryInt(fsd, src);
@@ -249,7 +252,7 @@ class FSDirStatAndListingOp {
       Snapshot.Root sRoot = snapshots.get(i + skipSize).getRoot();
       listing[i] = createFileStatus(fsd, sRoot.getLocalNameBytes(), sRoot,
           BlockStoragePolicySuite.ID_UNSPECIFIED, Snapshot.CURRENT_STATE_ID,
-          false, null);
+          false, INodesInPath.fromINode(sRoot));
     }
     return new DirectoryListing(
         listing, snapshots.size() - skipSize - numOfListing);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/475c6b49/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 950c9fe..444589e 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
@@ -476,7 +476,7 @@ public class FSDirectory implements Closeable {
   /**
    * This is a wrapper for resolvePath(). If the path passed
    * is prefixed with /.reserved/raw, then it checks to ensure that the caller
-   * has super user has super user privileges.
+   * has super user privileges.
    *
    * @param pc The permission checker used when resolving path.
    * @param path The path to resolve.
@@ -555,23 +555,23 @@ public class FSDirectory implements Closeable {
   }
 
   /** Set block storage policy for a directory */
-  void setStoragePolicy(String src, byte policyId)
+  void setStoragePolicy(INodesInPath iip, byte policyId)
       throws IOException {
     writeLock();
     try {
-      unprotectedSetStoragePolicy(src, policyId);
+      unprotectedSetStoragePolicy(iip, policyId);
     } finally {
       writeUnlock();
     }
   }
 
-  void unprotectedSetStoragePolicy(String src, byte policyId)
+  void unprotectedSetStoragePolicy(INodesInPath iip, byte policyId)
       throws IOException {
     assert hasWriteLock();
-    final INodesInPath iip = getINodesInPath4Write(src, true);
     final INode inode = iip.getLastINode();
     if (inode == null) {
-      throw new FileNotFoundException("File/Directory does not exist: " + src);
+      throw new FileNotFoundException("File/Directory does not exist: "
+          + iip.getPath());
     }
     final int snapshotId = iip.getLatestSnapshotId();
     if (inode.isFile()) {
@@ -593,7 +593,8 @@ public class FSDirectory implements Closeable {
     } else if (inode.isDirectory()) {
       setDirStoragePolicy(inode.asDirectory(), policyId, snapshotId);  
     } else {
-      throw new FileNotFoundException(src + " is not a file or directory");
+      throw new FileNotFoundException(iip.getPath()
+          + " is not a file or directory");
     }
   }
 
@@ -728,11 +729,11 @@ public class FSDirectory implements Closeable {
   /**
    * @return true if the path is a non-empty directory; otherwise, return false.
    */
-  boolean isNonEmptyDirectory(String path) throws UnresolvedLinkException {
+  boolean isNonEmptyDirectory(INodesInPath inodesInPath) {
     readLock();
     try {
-      final INodesInPath inodesInPath = getLastINodeInPath(path, false);
-      final INode inode = inodesInPath.getINode(0);
+      final INode[] inodes = inodesInPath.getINodes();
+      final INode inode = inodes[inodes.length - 1];
       if (inode == null || !inode.isDirectory()) {
         //not found or not a directory
         return false;
@@ -825,7 +826,7 @@ public class FSDirectory implements Closeable {
     }
     if (NameNode.stateChangeLog.isDebugEnabled()) {
       NameNode.stateChangeLog.debug("DIR* FSDirectory.unprotectedDelete: "
-          + targetNode.getFullPathName() + " is removed");
+          + iip.getPath() + " is removed");
     }
     return removed;
   }
@@ -1858,9 +1859,6 @@ public class FSDirectory implements Closeable {
     }
     readLock();
     try {
-      if (iip == null) {
-        iip = getINodesInPath(inode.getFullPathName(), true);
-      }
       EncryptionZone encryptionZone = getEZForPath(iip);
       if (encryptionZone == null) {
         // not an encrypted file
@@ -1882,8 +1880,7 @@ public class FSDirectory implements Closeable {
 
       if (fileXAttr == null) {
         NameNode.LOG.warn("Could not find encryption XAttr for file " +
-            inode.getFullPathName() + " in encryption zone " +
-            encryptionZone.getPath());
+            iip.getPath() + " in encryption zone " + encryptionZone.getPath());
         return null;
       }
 
@@ -2307,31 +2304,28 @@ public class FSDirectory implements Closeable {
     }
   }
 
-  void checkOwner(FSPermissionChecker pc, String path)
-      throws AccessControlException, UnresolvedLinkException {
-    checkPermission(pc, path, true, null, null, null, null);
+  void checkOwner(FSPermissionChecker pc, INodesInPath iip)
+      throws AccessControlException {
+    checkPermission(pc, iip, true, null, null, null, null);
   }
 
-  void checkPathAccess(FSPermissionChecker pc, String path,
-                       FsAction access)
-      throws AccessControlException, UnresolvedLinkException {
-    checkPermission(pc, path, false, null, null, access, null);
+  void checkPathAccess(FSPermissionChecker pc, INodesInPath iip,
+      FsAction access) throws AccessControlException {
+    checkPermission(pc, iip, false, null, null, access, null);
   }
-  void checkParentAccess(
-      FSPermissionChecker pc, String path, FsAction access)
-      throws AccessControlException, UnresolvedLinkException {
-    checkPermission(pc, path, false, null, access, null, null);
+  void checkParentAccess(FSPermissionChecker pc, INodesInPath iip,
+      FsAction access) throws AccessControlException {
+    checkPermission(pc, iip, false, null, access, null, null);
   }
 
-  void checkAncestorAccess(
-      FSPermissionChecker pc, String path, FsAction access)
-      throws AccessControlException, UnresolvedLinkException {
-    checkPermission(pc, path, false, access, null, null, null);
+  void checkAncestorAccess(FSPermissionChecker pc, INodesInPath iip,
+      FsAction access) throws AccessControlException {
+    checkPermission(pc, iip, false, access, null, null, null);
   }
 
-  void checkTraverse(FSPermissionChecker pc, String path)
-      throws AccessControlException, UnresolvedLinkException {
-    checkPermission(pc, path, false, null, null, null, null);
+  void checkTraverse(FSPermissionChecker pc, INodesInPath iip)
+      throws AccessControlException {
+    checkPermission(pc, iip, false, null, null, null, null);
   }
 
   /**
@@ -2339,13 +2333,12 @@ public class FSDirectory implements Closeable {
    * details of the parameters, see
    * {@link FSPermissionChecker#checkPermission}.
    */
-  void checkPermission(
-      FSPermissionChecker pc, String path, boolean doCheckOwner,
-      FsAction ancestorAccess, FsAction parentAccess, FsAction access,
-      FsAction subAccess)
-    throws AccessControlException, UnresolvedLinkException {
-    checkPermission(pc, path, doCheckOwner, ancestorAccess,
-        parentAccess, access, subAccess, false, true);
+  void checkPermission(FSPermissionChecker pc, INodesInPath iip,
+      boolean doCheckOwner, FsAction ancestorAccess, FsAction parentAccess,
+      FsAction access, FsAction subAccess)
+    throws AccessControlException {
+    checkPermission(pc, iip, doCheckOwner, ancestorAccess,
+        parentAccess, access, subAccess, false);
   }
 
   /**
@@ -2353,16 +2346,15 @@ public class FSDirectory implements Closeable {
    * details of the parameters, see
    * {@link FSPermissionChecker#checkPermission}.
    */
-  void checkPermission(
-      FSPermissionChecker pc, String path, boolean doCheckOwner,
-      FsAction ancestorAccess, FsAction parentAccess, FsAction access,
-      FsAction subAccess, boolean ignoreEmptyDir, boolean resolveLink)
-      throws AccessControlException, UnresolvedLinkException {
+  void checkPermission(FSPermissionChecker pc, INodesInPath iip,
+      boolean doCheckOwner, FsAction ancestorAccess, FsAction parentAccess,
+      FsAction access, FsAction subAccess, boolean ignoreEmptyDir)
+      throws AccessControlException {
     if (!pc.isSuperUser()) {
       readLock();
       try {
-        pc.checkPermission(path, this, doCheckOwner, ancestorAccess,
-            parentAccess, access, subAccess, ignoreEmptyDir, resolveLink);
+        pc.checkPermission(iip, doCheckOwner, ancestorAccess,
+            parentAccess, access, subAccess, ignoreEmptyDir);
       } finally {
         readUnlock();
       }
@@ -2379,12 +2371,11 @@ public class FSDirectory implements Closeable {
   /**
    * Verify that parent directory of src exists.
    */
-  void verifyParentDir(String src)
-      throws FileNotFoundException, ParentNotDirectoryException,
-             UnresolvedLinkException {
+  void verifyParentDir(INodesInPath iip, String src)
+      throws FileNotFoundException, ParentNotDirectoryException {
     Path parent = new Path(src).getParent();
     if (parent != null) {
-      final INode parentNode = getINode(parent.toString());
+      final INode parentNode = iip.getINode(-2);
       if (parentNode == null) {
         throw new FileNotFoundException("Parent directory doesn't exist: "
             + parent);
@@ -2407,7 +2398,6 @@ public class FSDirectory implements Closeable {
 
   /**
    * Set the last allocated inode id when fsimage or editlog is loaded.
-   * @param newValue
    */
   void resetLastInodeId(long newValue) throws IOException {
     try {
@@ -2417,8 +2407,7 @@ public class FSDirectory implements Closeable {
     }
   }
 
-  /** Should only be used for tests to reset to any value
-   * @param newValue*/
+  /** Should only be used for tests to reset to any value */
   void resetLastInodeIdWithoutChecking(long newValue) {
     inodeId.setCurrentValue(newValue);
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/475c6b49/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java
index c33477a..f60f142 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java
@@ -664,7 +664,8 @@ public class FSEditLogLoader {
       final String snapshotRoot =
           renameReservedPathsOnUpgrade(createSnapshotOp.snapshotRoot,
               logVersion);
-      String path = fsNamesys.getSnapshotManager().createSnapshot(
+      INodesInPath iip = fsDir.getINodesInPath4Write(snapshotRoot);
+      String path = fsNamesys.getSnapshotManager().createSnapshot(iip,
           snapshotRoot, createSnapshotOp.snapshotName);
       if (toAddRetryCache) {
         fsNamesys.addCacheEntryWithPayload(createSnapshotOp.rpcClientId,
@@ -679,8 +680,9 @@ public class FSEditLogLoader {
       final String snapshotRoot =
           renameReservedPathsOnUpgrade(deleteSnapshotOp.snapshotRoot,
               logVersion);
+      INodesInPath iip = fsDir.getINodesInPath4Write(snapshotRoot);
       fsNamesys.getSnapshotManager().deleteSnapshot(
-          snapshotRoot, deleteSnapshotOp.snapshotName,
+          iip, deleteSnapshotOp.snapshotName,
           collectedBlocks, removedINodes);
       fsNamesys.removeBlocksAndUpdateSafemodeTotal(collectedBlocks);
       collectedBlocks.clear();
@@ -698,7 +700,8 @@ public class FSEditLogLoader {
       final String snapshotRoot =
           renameReservedPathsOnUpgrade(renameSnapshotOp.snapshotRoot,
               logVersion);
-      fsNamesys.getSnapshotManager().renameSnapshot(
+      INodesInPath iip = fsDir.getINodesInPath4Write(snapshotRoot);
+      fsNamesys.getSnapshotManager().renameSnapshot(iip,
           snapshotRoot, renameSnapshotOp.snapshotOldName,
           renameSnapshotOp.snapshotNewName);
       
@@ -844,9 +847,10 @@ public class FSEditLogLoader {
     }
     case OP_SET_STORAGE_POLICY: {
       SetStoragePolicyOp setStoragePolicyOp = (SetStoragePolicyOp) op;
-      fsDir.unprotectedSetStoragePolicy(
-          renameReservedPathsOnUpgrade(setStoragePolicyOp.path, logVersion),
-          setStoragePolicyOp.policyId);
+      final String path = renameReservedPathsOnUpgrade(setStoragePolicyOp.path,
+          logVersion);
+      final INodesInPath iip = fsDir.getINodesInPath4Write(path);
+      fsDir.unprotectedSetStoragePolicy(iip, setStoragePolicyOp.policyId);
       break;
     }
     default:

http://git-wip-us.apache.org/repos/asf/hadoop/blob/475c6b49/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 22039fc..0a558dd 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
@@ -1664,11 +1664,6 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     pw.flush();
     return sw.toString();
   }
-  
-
-  long getDefaultBlockSize() {
-    return serverDefaults.getBlockSize();
-  }
 
   FsServerDefaults getServerDefaults() throws StandbyException {
     checkOperation(OperationCategory.READ);
@@ -1692,9 +1687,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
    * Set permissions for an existing file.
    * @throws IOException
    */
-  void setPermission(String src, FsPermission permission)
-      throws AccessControlException, FileNotFoundException, SafeModeException,
-      UnresolvedLinkException, IOException {
+  void setPermission(String src, FsPermission permission) throws IOException {
     try {
       setPermissionInt(src, permission);
     } catch (AccessControlException e) {
@@ -1704,8 +1697,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
   }
 
   private void setPermissionInt(final String srcArg, FsPermission permission)
-      throws AccessControlException, FileNotFoundException, SafeModeException,
-      UnresolvedLinkException, IOException {
+      throws IOException {
     String src = srcArg;
     HdfsFileStatus resultingStat = null;
     FSPermissionChecker pc = getPermissionChecker();
@@ -1716,7 +1708,8 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
       checkOperation(OperationCategory.WRITE);
       checkNameNodeSafeMode("Cannot set permission for " + src);
       src = dir.resolvePath(pc, src, pathComponents);
-      checkOwner(pc, src);
+      final INodesInPath iip = dir.getINodesInPath4Write(src);
+      dir.checkOwner(pc, iip);
       dir.setPermission(src, permission);
       getEditLog().logSetPermissions(src, permission);
       resultingStat = getAuditFileInfo(src, false);
@@ -1732,8 +1725,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
    * @throws IOException
    */
   void setOwner(String src, String username, String group)
-      throws AccessControlException, FileNotFoundException, SafeModeException,
-      UnresolvedLinkException, IOException {
+      throws IOException {
     try {
       setOwnerInt(src, username, group);
     } catch (AccessControlException e) {
@@ -1743,8 +1735,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
   }
 
   private void setOwnerInt(final String srcArg, String username, String group)
-      throws AccessControlException, FileNotFoundException, SafeModeException,
-      UnresolvedLinkException, IOException {
+      throws IOException {
     String src = srcArg;
     HdfsFileStatus resultingStat = null;
     FSPermissionChecker pc = getPermissionChecker();
@@ -1755,7 +1746,8 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
       checkOperation(OperationCategory.WRITE);
       checkNameNodeSafeMode("Cannot set owner for " + src);
       src = dir.resolvePath(pc, src, pathComponents);
-      checkOwner(pc, src);
+      final INodesInPath iip = dir.getINodesInPath4Write(src);
+      dir.checkOwner(pc, iip);
       if (!pc.isSuperUser()) {
         if (username != null && !pc.getUser().equals(username)) {
           throw new AccessControlException("Non-super user cannot change owner");
@@ -1856,8 +1848,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
    */
   private LocatedBlocks getBlockLocationsUpdateTimes(final String srcArg,
       long offset, long length, boolean doAccessTime, boolean needBlockToken)
-      throws FileNotFoundException,
-      UnresolvedLinkException, IOException {
+      throws IOException {
     String src = srcArg;
     FSPermissionChecker pc = getPermissionChecker();
     byte[][] pathComponents = FSDirectory.getPathComponentsForReservedPath(src);
@@ -1871,14 +1862,15 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
         writeLock(); // writelock is needed to set accesstime
       }
       try {
-        src = dir.resolvePath(pc, src, pathComponents);
         if (isReadOp) {
           checkOperation(OperationCategory.READ);
         } else {
           checkOperation(OperationCategory.WRITE);
         }
+        src = dir.resolvePath(pc, src, pathComponents);
+        final INodesInPath iip = dir.getINodesInPath(src, true);
         if (isPermissionEnabled) {
-          checkPathAccess(pc, src, FsAction.READ);
+          dir.checkPathAccess(pc, iip, FsAction.READ);
         }
 
         // if the namenode is in safemode, then do not update access time
@@ -1886,7 +1878,6 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
           doAccessTime = false;
         }
 
-        final INodesInPath iip = dir.getINodesInPath(src, true);
         final INode[] inodes = iip.getINodes();
         final INodeFile inode = INodeFile.valueOf(
             inodes[inodes.length - 1], src);
@@ -1995,7 +1986,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
   }
 
   private void setTimesInt(final String srcArg, long mtime, long atime)
-    throws IOException, UnresolvedLinkException {
+    throws IOException {
     String src = srcArg;
     HdfsFileStatus resultingStat = null;
     FSPermissionChecker pc = getPermissionChecker();
@@ -2006,12 +1997,11 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
       checkOperation(OperationCategory.WRITE);
       checkNameNodeSafeMode("Cannot set times " + src);
       src = dir.resolvePath(pc, src, pathComponents);
-
+      final INodesInPath iip = dir.getINodesInPath4Write(src);
       // Write access is required to set access and modification times
       if (isPermissionEnabled) {
-        checkPathAccess(pc, src, FsAction.WRITE);
+        dir.checkPathAccess(pc, iip, FsAction.WRITE);
       }
-      final INodesInPath iip = dir.getINodesInPath4Write(src);
       final INode inode = iip.getLastINode();
       if (inode != null) {
         boolean changed = dir.setTimes(inode, mtime, atime, true,
@@ -2034,7 +2024,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
    */
   void createSymlink(String target, String link,
       PermissionStatus dirPerms, boolean createParent, boolean logRetryCache)
-      throws IOException, UnresolvedLinkException {
+      throws IOException {
     if (!DFSUtil.isValidName(link)) {
       throw new InvalidPathException("Invalid link name: " + link);
     }
@@ -2042,10 +2032,8 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
       throw new InvalidPathException("Invalid target name: " + target);
     }
 
-    boolean success = false;
     try {
       createSymlinkInt(target, link, dirPerms, createParent, logRetryCache);
-      success = true;
     } catch (AccessControlException e) {
       logAuditEvent(false, "createSymlink", link, target, null);
       throw e;
@@ -2054,7 +2042,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
 
   private void createSymlinkInt(String target, final String linkArg,
       PermissionStatus dirPerms, boolean createParent, boolean logRetryCache) 
-      throws IOException, UnresolvedLinkException {
+      throws IOException {
     String link = linkArg;
     if (NameNode.stateChangeLog.isDebugEnabled()) {
       NameNode.stateChangeLog.debug("DIR* NameSystem.createSymlink: target="
@@ -2069,15 +2057,16 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
       checkOperation(OperationCategory.WRITE);
       checkNameNodeSafeMode("Cannot create symlink " + link);
       link = dir.resolvePath(pc, link, pathComponents);
+      final INodesInPath iip = dir.getINodesInPath4Write(link, false);
       if (!createParent) {
-        dir.verifyParentDir(link);
+        dir.verifyParentDir(iip, link);
       }
       if (!dir.isValidToCreate(link)) {
         throw new IOException("failed to create link " + link 
             +" either because the filename is invalid or the file exists");
       }
       if (isPermissionEnabled) {
-        checkAncestorAccess(pc, link, FsAction.WRITE);
+        dir.checkAncestorAccess(pc, iip, FsAction.WRITE);
       }
       // validate that we have enough inodes.
       checkFsObjectLimit();
@@ -2129,8 +2118,9 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
       checkOperation(OperationCategory.WRITE);
       checkNameNodeSafeMode("Cannot set replication for " + src);
       src = dir.resolvePath(pc, src, pathComponents);
+      final INodesInPath iip = dir.getINodesInPath4Write(src);
       if (isPermissionEnabled) {
-        checkPathAccess(pc, src, FsAction.WRITE);
+        dir.checkPathAccess(pc, iip, FsAction.WRITE);
       }
 
       final short[] blockRepls = new short[2]; // 0: old, 1: new
@@ -2168,8 +2158,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
   }
 
   private void setStoragePolicyInt(String src, final String policyName)
-      throws IOException, UnresolvedLinkException, AccessControlException {
-
+      throws IOException {
     if (!isStoragePolicyEnabled) {
       throw new IOException("Failed to set storage policy since "
           + DFS_STORAGE_POLICY_ENABLED_KEY + " is set to false.");
@@ -2188,20 +2177,20 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
       checkOperation(OperationCategory.WRITE);
       checkNameNodeSafeMode("Cannot set storage policy for " + src);
 
+      src = FSDirectory.resolvePath(src, pathComponents, dir);
+      final INodesInPath iip = dir.getINodesInPath4Write(src);
+
       if (pc != null) {
-        checkPermission(pc, src, false, null, null, FsAction.WRITE, null,
-                        false, true);
+        dir.checkPermission(pc, iip, false, null, null, FsAction.WRITE, null, false);
       }
 
-      src = FSDirectory.resolvePath(src, pathComponents, dir);
-
       // get the corresponding policy and make sure the policy name is valid
       BlockStoragePolicy policy = blockManager.getStoragePolicy(policyName);
       if (policy == null) {
         throw new HadoopIllegalArgumentException(
             "Cannot find a block policy with the name " + policyName);
       }
-      dir.setStoragePolicy(src, policy.getId());
+      dir.setStoragePolicy(iip, policy.getId());
       getEditLog().logSetStoragePolicy(src, policy.getId());
       fileStat = getAuditFileInfo(src, false);
     } finally {
@@ -2227,8 +2216,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     }
   }
 
-  long getPreferredBlockSize(String filename) 
-      throws IOException, UnresolvedLinkException {
+  long getPreferredBlockSize(String filename) throws IOException {
     FSPermissionChecker pc = getPermissionChecker();
     checkOperation(OperationCategory.READ);
     byte[][] pathComponents = FSDirectory.getPathComponentsForReservedPath(filename);
@@ -2236,8 +2224,9 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     try {
       checkOperation(OperationCategory.READ);
       filename = dir.resolvePath(pc, filename, pathComponents);
+      final INodesInPath iip = dir.getINodesInPath(filename, true);
       if (isPermissionEnabled) {
-        checkTraverse(pc, filename);
+        dir.checkTraverse(pc, iip);
       }
       return dir.getPreferredBlockSize(filename);
     } finally {
@@ -2438,7 +2427,8 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
       checkOperation(OperationCategory.WRITE);
       checkNameNodeSafeMode("Cannot create file" + src);
       src = dir.resolvePath(pc, src, pathComponents);
-      toRemoveBlocks = startFileInternal(pc, src, permissions, holder, 
+      final INodesInPath iip = dir.getINodesInPath4Write(src);
+      toRemoveBlocks = startFileInternal(pc, iip, permissions, holder,
           clientMachine, create, overwrite, createParent, replication, 
           blockSize, isLazyPersist, suite, protocolVersion, edek, logRetryCache);
       stat = FSDirStatAndListingOp.getFileInfo(dir, src, false,
@@ -2473,23 +2463,36 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
    * {@link ClientProtocol#create}
    */
   private BlocksMapUpdateInfo startFileInternal(FSPermissionChecker pc, 
-      String src, PermissionStatus permissions, String holder, 
+      INodesInPath iip, PermissionStatus permissions, String holder,
       String clientMachine, boolean create, boolean overwrite, 
       boolean createParent, short replication, long blockSize, 
       boolean isLazyPersist, CipherSuite suite, CryptoProtocolVersion version,
       EncryptedKeyVersion edek, boolean logRetryEntry)
-      throws FileAlreadyExistsException, AccessControlException,
-      UnresolvedLinkException, FileNotFoundException,
-      ParentNotDirectoryException, RetryStartFileException, IOException {
+      throws IOException {
     assert hasWriteLock();
     // Verify that the destination does not exist as a directory already.
-    final INodesInPath iip = dir.getINodesInPath4Write(src);
     final INode inode = iip.getLastINode();
+    final String src = iip.getPath();
     if (inode != null && inode.isDirectory()) {
       throw new FileAlreadyExistsException(src +
           " already exists as a directory");
     }
 
+    final INodeFile myFile = INodeFile.valueOf(inode, src, true);
+    if (isPermissionEnabled) {
+      if (overwrite && myFile != null) {
+        dir.checkPathAccess(pc, iip, FsAction.WRITE);
+      }
+      /*
+       * To overwrite existing file, need to check 'w' permission 
+       * of parent (equals to ancestor in this case)
+       */
+      dir.checkAncestorAccess(pc, iip, FsAction.WRITE);
+    }
+    if (!createParent) {
+      dir.verifyParentDir(iip, src);
+    }
+
     FileEncryptionInfo feInfo = null;
 
     final EncryptionZone zone = dir.getEZForPath(iip);
@@ -2510,22 +2513,6 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
           ezKeyName, edek.getEncryptionKeyVersionName());
     }
 
-    final INodeFile myFile = INodeFile.valueOf(inode, src, true);
-    if (isPermissionEnabled) {
-      if (overwrite && myFile != null) {
-        checkPathAccess(pc, src, FsAction.WRITE);
-      }
-      /*
-       * To overwrite existing file, need to check 'w' permission 
-       * of parent (equals to ancestor in this case)
-       */
-      checkAncestorAccess(pc, src, FsAction.WRITE);
-    }
-
-    if (!createParent) {
-      dir.verifyParentDir(src);
-    }
-
     try {
       BlocksMapUpdateInfo toRemoveBlocks = null;
       if (myFile == null) {
@@ -2637,20 +2624,19 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
    * 
    * @return the last block locations if the block is partial or null otherwise
    */
-  private LocatedBlock appendFileInternal(FSPermissionChecker pc, String src,
-      String holder, String clientMachine, boolean logRetryCache)
-      throws AccessControlException, UnresolvedLinkException,
-      FileNotFoundException, IOException {
+  private LocatedBlock appendFileInternal(FSPermissionChecker pc,
+      INodesInPath iip, String holder, String clientMachine,
+      boolean logRetryCache) throws IOException {
     assert hasWriteLock();
     // Verify that the destination does not exist as a directory already.
-    final INodesInPath iip = dir.getINodesInPath4Write(src);
     final INode inode = iip.getLastINode();
+    final String src = iip.getPath();
     if (inode != null && inode.isDirectory()) {
       throw new FileAlreadyExistsException("Cannot append to directory " + src
           + "; already exists as a directory.");
     }
     if (isPermissionEnabled) {
-      checkPathAccess(pc, src, FsAction.WRITE);
+      dir.checkPathAccess(pc, iip, FsAction.WRITE);
     }
 
     try {
@@ -2754,12 +2740,13 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
       checkOperation(OperationCategory.WRITE);
       checkNameNodeSafeMode("Cannot recover the lease of " + src);
       src = dir.resolvePath(pc, src, pathComponents);
-      final INodeFile inode = INodeFile.valueOf(dir.getINode(src), src);
+      final INodesInPath iip = dir.getINodesInPath4Write(src);
+      final INodeFile inode = INodeFile.valueOf(iip.getLastINode(), src);
       if (!inode.isUnderConstruction()) {
         return true;
       }
       if (isPermissionEnabled) {
-        checkPathAccess(pc, src, FsAction.WRITE);
+        dir.checkPathAccess(pc, iip, FsAction.WRITE);
       }
   
       recoverLeaseInternal(inode, src, holder, clientMachine, true);
@@ -2889,7 +2876,8 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
       checkOperation(OperationCategory.WRITE);
       checkNameNodeSafeMode("Cannot append to file" + src);
       src = dir.resolvePath(pc, src, pathComponents);
-      lb = appendFileInternal(pc, src, holder, clientMachine, logRetryCache);
+      final INodesInPath iip = dir.getINodesInPath4Write(src);
+      lb = appendFileInternal(pc, iip, holder, clientMachine, logRetryCache);
       stat = FSDirStatAndListingOp.getFileInfo(dir, src, false,
           FSDirectory.isReservedRawName(srcArg), true);
     } catch (StandbyException se) {
@@ -3525,7 +3513,8 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
 
   /** 
    * Change the indicated filename. 
-   * @deprecated Use {@link #renameTo(String, String, Options.Rename...)} instead.
+   * @deprecated Use {@link #renameTo(String, String, boolean,
+   * Options.Rename...)} instead.
    */
   @Deprecated
   boolean renameTo(String src, String dst, boolean logRetryCache)
@@ -3651,12 +3640,13 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
       checkOperation(OperationCategory.WRITE);
       checkNameNodeSafeMode("Cannot delete " + src);
       src = dir.resolvePath(pc, src, pathComponents);
-      if (!recursive && dir.isNonEmptyDirectory(src)) {
+      final INodesInPath iip = dir.getINodesInPath4Write(src, false);
+      if (!recursive && dir.isNonEmptyDirectory(iip)) {
         throw new PathIsNotEmptyDirectoryException(src + " is non empty");
       }
       if (enforcePermission && isPermissionEnabled) {
-        checkPermission(pc, src, false, null, FsAction.WRITE, null,
-            FsAction.ALL, true, false);
+        dir.checkPermission(pc, iip, false, null, FsAction.WRITE, null,
+            FsAction.ALL, true);
       }
 
       long mtime = now();
@@ -3794,7 +3784,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
   /**
    * Get the file info for a specific file.
    *
-   * @param srcArg The string representation of the path to the file
+   * @param src The string representation of the path to the file
    * @param resolveLink whether to throw UnresolvedLinkException
    *        if src refers to a symlink
    *
@@ -5834,17 +5824,6 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     return new PermissionStatus(fsOwner.getShortUserName(), supergroup, permission);
   }
 
-  private void checkOwner(FSPermissionChecker pc, String path)
-      throws AccessControlException, UnresolvedLinkException {
-    dir.checkOwner(pc, path);
-  }
-
-  private void checkPathAccess(FSPermissionChecker pc,
-      String path, FsAction access) throws AccessControlException,
-      UnresolvedLinkException {
-    dir.checkPathAccess(pc, path, access);
-  }
-
   private void checkUnreadableBySuperuser(FSPermissionChecker pc,
       INode inode, int snapshotId)
       throws IOException {
@@ -5860,23 +5839,6 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     }
   }
 
-  private void checkParentAccess(FSPermissionChecker pc,
-      String path, FsAction access) throws AccessControlException,
-      UnresolvedLinkException {
-    dir.checkParentAccess(pc, path, access);
-  }
-
-  private void checkAncestorAccess(FSPermissionChecker pc,
-      String path, FsAction access) throws AccessControlException,
-      UnresolvedLinkException {
-    dir.checkAncestorAccess(pc, path, access);
-  }
-
-  private void checkTraverse(FSPermissionChecker pc, String path)
-      throws AccessControlException, UnresolvedLinkException {
-    dir.checkTraverse(pc, path);
-  }
-
   @Override
   public void checkSuperuserPrivilege()
       throws AccessControlException {
@@ -5887,28 +5849,6 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
   }
 
   /**
-   * Check whether current user have permissions to access the path. For more
-   * details of the parameters, see
-   * {@link FSPermissionChecker#checkPermission}.
-   */
-  private void checkPermission(FSPermissionChecker pc,
-      String path, boolean doCheckOwner, FsAction ancestorAccess,
-      FsAction parentAccess, FsAction access, FsAction subAccess)
-      throws AccessControlException, UnresolvedLinkException {
-    checkPermission(pc, path, doCheckOwner, ancestorAccess,
-            parentAccess, access, subAccess, false, true);
-  }
-
-  private void checkPermission(FSPermissionChecker pc,
-      String path, boolean doCheckOwner, FsAction ancestorAccess,
-      FsAction parentAccess, FsAction access, FsAction subAccess,
-      boolean ignoreEmptyDir, boolean resolveLink)
-      throws AccessControlException, UnresolvedLinkException {
-    dir.checkPermission(pc, path, doCheckOwner, ancestorAccess, parentAccess,
-        access, subAccess, ignoreEmptyDir, resolveLink);
-  }
-  
-  /**
    * Check to see if we have exceeded the limit on the number
    * of inodes.
    */
@@ -6299,10 +6239,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
         + newBlock.getLocalBlock() + ") success");
   }
 
-  /**
-   * @see #updatePipeline(String,  ExtendedBlock, ExtendedBlock, DatanodeID[], String[])
-   */
-  private void updatePipelineInternal(String clientName, ExtendedBlock oldBlock, 
+  private void updatePipelineInternal(String clientName, ExtendedBlock oldBlock,
       ExtendedBlock newBlock, DatanodeID[] newNodes, String[] newStorageIDs,
       boolean logRetryCache)
       throws IOException {
@@ -7387,9 +7324,8 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
    * @throws SafeModeException
    * @throws IOException
    */
-  void deleteSnapshot(
-      String snapshotRoot, String snapshotName, boolean logRetryCache)
-      throws IOException {
+  void deleteSnapshot(String snapshotRoot, String snapshotName,
+      boolean logRetryCache) throws IOException {
     checkOperation(OperationCategory.WRITE);
     boolean success = false;
     writeLock();
@@ -7397,6 +7333,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     try {
       checkOperation(OperationCategory.WRITE);
       checkNameNodeSafeMode("Cannot delete snapshot for " + snapshotRoot);
+
       blocksToBeDeleted = FSDirSnapshotOp.deleteSnapshot(dir, snapshotManager,
           snapshotRoot, snapshotName, logRetryCache);
       success = true;
@@ -7855,7 +7792,8 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
       checkOperation(OperationCategory.WRITE);
       checkNameNodeSafeMode("Cannot modify ACL entries on " + src);
       src = dir.resolvePath(pc, src, pathComponents);
-      checkOwner(pc, src);
+      final INodesInPath iip = dir.getINodesInPath4Write(src);
+      dir.checkOwner(pc, iip);
       List<AclEntry> newAcl = dir.modifyAclEntries(src, aclSpec);
       getEditLog().logSetAcl(src, newAcl);
       resultingStat = getAuditFileInfo(src, false);
@@ -7882,7 +7820,8 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
       checkOperation(OperationCategory.WRITE);
       checkNameNodeSafeMode("Cannot remove ACL entries on " + src);
       src = dir.resolvePath(pc, src, pathComponents);
-      checkOwner(pc, src);
+      final INodesInPath iip = dir.getINodesInPath4Write(src);
+      dir.checkOwner(pc, iip);
       List<AclEntry> newAcl = dir.removeAclEntries(src, aclSpec);
       getEditLog().logSetAcl(src, newAcl);
       resultingStat = getAuditFileInfo(src, false);
@@ -7908,7 +7847,8 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
       checkOperation(OperationCategory.WRITE);
       checkNameNodeSafeMode("Cannot remove default ACL entries on " + src);
       src = dir.resolvePath(pc, src, pathComponents);
-      checkOwner(pc, src);
+      final INodesInPath iip = dir.getINodesInPath4Write(src);
+      dir.checkOwner(pc, iip);
       List<AclEntry> newAcl = dir.removeDefaultAcl(src);
       getEditLog().logSetAcl(src, newAcl);
       resultingStat = getAuditFileInfo(src, false);
@@ -7934,7 +7874,8 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
       checkOperation(OperationCategory.WRITE);
       checkNameNodeSafeMode("Cannot remove ACL on " + src);
       src = dir.resolvePath(pc, src, pathComponents);
-      checkOwner(pc, src);
+      final INodesInPath iip = dir.getINodesInPath4Write(src);
+      dir.checkOwner(pc, iip);
       dir.removeAcl(src);
       getEditLog().logSetAcl(src, AclFeature.EMPTY_ENTRY_LIST);
       resultingStat = getAuditFileInfo(src, false);
@@ -7960,7 +7901,8 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
       checkOperation(OperationCategory.WRITE);
       checkNameNodeSafeMode("Cannot set ACL on " + src);
       src = dir.resolvePath(pc, src, pathComponents);
-      checkOwner(pc, src);
+      final INodesInPath iip = dir.getINodesInPath4Write(src);
+      dir.checkOwner(pc, iip);
       List<AclEntry> newAcl = dir.setAcl(src, aclSpec);
       getEditLog().logSetAcl(src, newAcl);
       resultingStat = getAuditFileInfo(src, false);
@@ -7984,8 +7926,9 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     try {
       checkOperation(OperationCategory.READ);
       src = dir.resolvePath(pc, src, pathComponents);
+      INodesInPath iip = dir.getINodesInPath(src, true);
       if (isPermissionEnabled) {
-        checkPermission(pc, src, false, null, null, null, null);
+        dir.checkPermission(pc, iip, false, null, null, null, null);
       }
       final AclStatus ret = dir.getAclStatus(src);
       success = true;
@@ -8095,12 +8038,12 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     checkOperation(OperationCategory.READ);
     readLock();
     try {
-      if (isPermissionEnabled) {
-        checkPathAccess(pc, src, FsAction.READ);
-      }
       checkOperation(OperationCategory.READ);
       src = dir.resolvePath(pc, src, pathComponents);
       final INodesInPath iip = dir.getINodesInPath(src, true);
+      if (isPermissionEnabled) {
+        dir.checkPathAccess(pc, iip, FsAction.READ);
+      }
       final EncryptionZone ret = dir.getEZForPath(iip);
       resultingStat = getAuditFileInfo(src, false);
       success = true;
@@ -8172,7 +8115,8 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
       checkOperation(OperationCategory.WRITE);
       checkNameNodeSafeMode("Cannot set XAttr on " + src);
       src = dir.resolvePath(pc, src, pathComponents);
-      checkXAttrChangeAccess(src, xAttr, pc);
+      final INodesInPath iip = dir.getINodesInPath4Write(src);
+      checkXAttrChangeAccess(iip, xAttr, pc);
       List<XAttr> xAttrs = Lists.newArrayListWithCapacity(1);
       xAttrs.add(xAttr);
       dir.setXAttrs(src, xAttrs, flag);
@@ -8224,10 +8168,11 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     byte[][] pathComponents = FSDirectory.getPathComponentsForReservedPath(src);
     readLock();
     try {
-      src = dir.resolvePath(pc, src, pathComponents);
       checkOperation(OperationCategory.READ);
+      src = dir.resolvePath(pc, src, pathComponents);
+      final INodesInPath iip = dir.getINodesInPath(src, true);
       if (isPermissionEnabled) {
-        checkPathAccess(pc, src, FsAction.READ);
+        dir.checkPathAccess(pc, iip, FsAction.READ);
       }
       List<XAttr> all = dir.getXAttrs(src);
       List<XAttr> filteredAll = XAttrPermissionFilter.
@@ -8272,16 +8217,16 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     byte[][] pathComponents = FSDirectory.getPathComponentsForReservedPath(src);
     readLock();
     try {
-      src = dir.resolvePath(pc, src, pathComponents);
       checkOperation(OperationCategory.READ);
+      src = dir.resolvePath(pc, src, pathComponents);
+      final INodesInPath iip = dir.getINodesInPath(src, true);
       if (isPermissionEnabled) {
         /* To access xattr names, you need EXECUTE in the owning directory. */
-        checkParentAccess(pc, src, FsAction.EXECUTE);
+        dir.checkParentAccess(pc, iip, FsAction.EXECUTE);
       }
       final List<XAttr> all = dir.getXAttrs(src);
-      final List<XAttr> filteredAll = XAttrPermissionFilter.
+      return XAttrPermissionFilter.
         filterXAttrsForApi(pc, all, isRawPath);
-      return filteredAll;
     } catch (AccessControlException e) {
       logAuditEvent(false, "listXAttrs", src);
       throw e;
@@ -8327,7 +8272,8 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
       checkOperation(OperationCategory.WRITE);
       checkNameNodeSafeMode("Cannot remove XAttr entry on " + src);
       src = dir.resolvePath(pc, src, pathComponents);
-      checkXAttrChangeAccess(src, xAttr, pc);
+      final INodesInPath iip = dir.getINodesInPath4Write(src);
+      checkXAttrChangeAccess(iip, xAttr, pc);
 
       List<XAttr> xAttrs = Lists.newArrayListWithCapacity(1);
       xAttrs.add(xAttr);
@@ -8346,37 +8292,37 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     logAuditEvent(true, "removeXAttr", srcArg, null, resultingStat);
   }
 
-  private void checkXAttrChangeAccess(String src, XAttr xAttr,
-      FSPermissionChecker pc) throws UnresolvedLinkException,
-      AccessControlException {
+  private void checkXAttrChangeAccess(INodesInPath iip, XAttr xAttr,
+      FSPermissionChecker pc) throws AccessControlException {
     if (isPermissionEnabled && xAttr.getNameSpace() == XAttr.NameSpace.USER) {
-      final INode inode = dir.getINode(src);
+      final INode inode = iip.getLastINode();
       if (inode != null &&
           inode.isDirectory() &&
           inode.getFsPermission().getStickyBit()) {
         if (!pc.isSuperUser()) {
-          checkOwner(pc, src);
+          dir.checkOwner(pc, iip);
         }
       } else {
-        checkPathAccess(pc, src, FsAction.WRITE);
+        dir.checkPathAccess(pc, iip, FsAction.WRITE);
       }
     }
   }
 
-  void checkAccess(String src, FsAction mode) throws AccessControlException,
-      FileNotFoundException, UnresolvedLinkException, IOException {
+  void checkAccess(String src, FsAction mode) throws IOException {
     checkOperation(OperationCategory.READ);
     byte[][] pathComponents = FSDirectory.getPathComponentsForReservedPath(src);
     readLock();
     try {
       checkOperation(OperationCategory.READ);
       src = FSDirectory.resolvePath(src, pathComponents, dir);
-      if (dir.getINode(src) == null) {
+      final INodesInPath iip = dir.getINodesInPath(src, true);
+      INode[] inodes = iip.getINodes();
+      if (inodes[inodes.length - 1] == null) {
         throw new FileNotFoundException("Path not found");
       }
       if (isPermissionEnabled) {
         FSPermissionChecker pc = getPermissionChecker();
-        checkPathAccess(pc, src, mode);
+        dir.checkPathAccess(pc, iip, mode);
       }
     } catch (AccessControlException e) {
       logAuditEvent(false, "checkAccess", src);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/475c6b49/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSPermissionChecker.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSPermissionChecker.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSPermissionChecker.java
index 54090f2..a0455dc 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSPermissionChecker.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSPermissionChecker.java
@@ -25,7 +25,6 @@ import java.util.Stack;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.fs.UnresolvedLinkException;
 import org.apache.hadoop.fs.permission.AclEntryScope;
 import org.apache.hadoop.fs.permission.AclEntryType;
 import org.apache.hadoop.fs.permission.FsAction;
@@ -58,18 +57,16 @@ class FSPermissionChecker {
     return sb.toString();
   }
 
-  private final UserGroupInformation ugi;
-  private final String user;  
+  private final String user;
   /** A set with group namess. Not synchronized since it is unmodifiable */
   private final Set<String> groups;
   private final boolean isSuper;
 
   FSPermissionChecker(String fsOwner, String supergroup,
       UserGroupInformation callerUgi) {
-    ugi = callerUgi;
-    HashSet<String> s = new HashSet<String>(Arrays.asList(ugi.getGroupNames()));
+    HashSet<String> s = new HashSet<String>(Arrays.asList(callerUgi.getGroupNames()));
     groups = Collections.unmodifiableSet(s);
-    user = ugi.getShortUserName();
+    user = callerUgi.getShortUserName();
     isSuper = user.equals(fsOwner) || groups.contains(supergroup);
   }
 
@@ -126,18 +123,15 @@ class FSPermissionChecker {
    * it is the access required of the path and all the sub-directories.
    * If path is not a directory, there is no effect.
    * @param ignoreEmptyDir Ignore permission checking for empty directory?
-   * @param resolveLink whether to resolve the final path component if it is
-   * a symlink
    * @throws AccessControlException
-   * @throws UnresolvedLinkException
    * 
    * Guarded by {@link FSNamesystem#readLock()}
    * Caller of this method must hold that lock.
    */
-  void checkPermission(String path, FSDirectory dir, boolean doCheckOwner,
+  void checkPermission(INodesInPath inodesInPath, boolean doCheckOwner,
       FsAction ancestorAccess, FsAction parentAccess, FsAction access,
-      FsAction subAccess, boolean ignoreEmptyDir, boolean resolveLink)
-      throws AccessControlException, UnresolvedLinkException {
+      FsAction subAccess, boolean ignoreEmptyDir)
+      throws AccessControlException {
     if (LOG.isDebugEnabled()) {
       LOG.debug("ACCESS CHECK: " + this
           + ", doCheckOwner=" + doCheckOwner
@@ -145,12 +139,10 @@ class FSPermissionChecker {
           + ", parentAccess=" + parentAccess
           + ", access=" + access
           + ", subAccess=" + subAccess
-          + ", ignoreEmptyDir=" + ignoreEmptyDir
-          + ", resolveLink=" + resolveLink);
+          + ", ignoreEmptyDir=" + ignoreEmptyDir);
     }
     // check if (parentAccess != null) && file exists, then check sb
     // If resolveLink, the check is performed on the link target.
-    final INodesInPath inodesInPath = dir.getINodesInPath(path, resolveLink);
     final int snapshotId = inodesInPath.getPathSnapshotId();
     final INode[] inodes = inodesInPath.getINodes();
     int ancestorIndex = inodes.length - 2;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/475c6b49/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodesInPath.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodesInPath.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodesInPath.java
index c74ebb0..58f5f3d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodesInPath.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodesInPath.java
@@ -41,8 +41,8 @@ public class INodesInPath {
    * @return true if path component is {@link HdfsConstants#DOT_SNAPSHOT_DIR}
    */
   private static boolean isDotSnapshotDir(byte[] pathComponent) {
-    return pathComponent == null ? false
-        : Arrays.equals(HdfsConstants.DOT_SNAPSHOT_DIR_BYTES, pathComponent);
+    return pathComponent != null &&
+        Arrays.equals(HdfsConstants.DOT_SNAPSHOT_DIR_BYTES, pathComponent);
   }
 
   static INodesInPath fromINode(INode inode) {
@@ -177,7 +177,7 @@ public class INodesInPath {
               (dstSnapshotId != Snapshot.CURRENT_STATE_ID && 
                 dstSnapshotId >= latest)) { // the above scenario 
             int lastSnapshot = Snapshot.CURRENT_STATE_ID;
-            DirectoryWithSnapshotFeature sf = null;
+            DirectoryWithSnapshotFeature sf;
             if (curNode.isDirectory() && 
                 (sf = curNode.asDirectory().getDirectoryWithSnapshotFeature()) != null) {
               lastSnapshot = sf.getLastSnapshotId();
@@ -186,7 +186,7 @@ public class INodesInPath {
           }
         }
       }
-      if (curNode.isSymlink() && (!lastComp || (lastComp && resolveLink))) {
+      if (curNode.isSymlink() && (!lastComp || resolveLink)) {
         final String path = constructPath(components, 0, components.length);
         final String preceding = constructPath(components, 0, count);
         final String remainder =
@@ -207,7 +207,7 @@ public class INodesInPath {
       final byte[] childName = components[count + 1];
       
       // check if the next byte[] in components is for ".snapshot"
-      if (isDotSnapshotDir(childName) && isDir && dir.isSnapshottable()) {
+      if (isDotSnapshotDir(childName) && dir.isSnapshottable()) {
         // skip the ".snapshot" in components
         count++;
         index++;
@@ -344,7 +344,12 @@ public class INodesInPath {
   byte[] getLastLocalName() {
     return path[path.length - 1];
   }
-  
+
+  /** @return the full path in string form */
+  public String getPath() {
+    return DFSUtil.byteArray2PathString(path);
+  }
+
   /**
    * @return index of the {@link Snapshot.Root} node in the inodes array,
    * -1 for non-snapshot paths.
@@ -398,7 +403,7 @@ public class INodesInPath {
 
   private String toString(boolean vaildateObject) {
     if (vaildateObject) {
-      vaildate();
+      validate();
     }
 
     final StringBuilder b = new StringBuilder(getClass().getSimpleName())
@@ -423,7 +428,7 @@ public class INodesInPath {
     return b.toString();
   }
 
-  void vaildate() {
+  void validate() {
     // check parent up to snapshotRootIndex or numNonNull
     final int n = snapshotRootIndex >= 0? snapshotRootIndex + 1: numNonNull;  
     int i = 0;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/475c6b49/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/SnapshotManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/SnapshotManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/SnapshotManager.java
index a808013..ba7c43c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/SnapshotManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/SnapshotManager.java
@@ -173,16 +173,15 @@ public class SnapshotManager implements SnapshotStatsMXBean {
   * Find the source root directory where the snapshot will be taken
   * for a given path.
   *
-  * @param path The directory path where the snapshot will be taken.
   * @return Snapshottable directory.
   * @throws IOException
   *           Throw IOException when the given path does not lead to an
   *           existing snapshottable directory.
   */
-  public INodeDirectory getSnapshottableRoot(final String path)
+  public INodeDirectory getSnapshottableRoot(final INodesInPath iip)
       throws IOException {
-    final INodeDirectory dir = INodeDirectory.valueOf(fsdir
-        .getINodesInPath4Write(path).getLastINode(), path);
+    final String path = iip.getPath();
+    final INodeDirectory dir = INodeDirectory.valueOf(iip.getLastINode(), path);
     if (!dir.isSnapshottable()) {
       throw new SnapshotException(
           "Directory is not a snapshottable directory: " + path);
@@ -194,8 +193,7 @@ public class SnapshotManager implements SnapshotStatsMXBean {
    * Create a snapshot of the given path.
    * It is assumed that the caller will perform synchronization.
    *
-   * @param path
-   *          The directory path where the snapshot will be taken.
+   * @param iip the INodes resolved from the snapshottable directory's path
    * @param snapshotName
    *          The name of the snapshot.
    * @throws IOException
@@ -204,9 +202,9 @@ public class SnapshotManager implements SnapshotStatsMXBean {
    *           snapshot with the given name for the directory, and/or 3)
    *           snapshot number exceeds quota
    */
-  public String createSnapshot(final String path, String snapshotName
-      ) throws IOException {
-    INodeDirectory srcRoot = getSnapshottableRoot(path);
+  public String createSnapshot(final INodesInPath iip, String snapshotRoot,
+      String snapshotName) throws IOException {
+    INodeDirectory srcRoot = getSnapshottableRoot(iip);
 
     if (snapshotCounter == getMaxSnapshotID()) {
       // We have reached the maximum allowable snapshot ID and since we don't
@@ -223,31 +221,25 @@ public class SnapshotManager implements SnapshotStatsMXBean {
     //create success, update id
     snapshotCounter++;
     numSnapshots.getAndIncrement();
-    return Snapshot.getSnapshotPath(path, snapshotName);
+    return Snapshot.getSnapshotPath(snapshotRoot, snapshotName);
   }
   
   /**
    * Delete a snapshot for a snapshottable directory
-   * @param path Path to the directory where the snapshot was taken
    * @param snapshotName Name of the snapshot to be deleted
    * @param collectedBlocks Used to collect information to update blocksMap 
    * @throws IOException
    */
-  public void deleteSnapshot(final String path, final String snapshotName,
+  public void deleteSnapshot(final INodesInPath iip, final String snapshotName,
       BlocksMapUpdateInfo collectedBlocks, final List<INode> removedINodes)
       throws IOException {
-    // parse the path, and check if the path is a snapshot path
-    // the INodeDirectorySnapshottable#valueOf method will throw Exception 
-    // if the path is not for a snapshottable directory
-    INodeDirectory srcRoot = getSnapshottableRoot(path);
+    INodeDirectory srcRoot = getSnapshottableRoot(iip);
     srcRoot.removeSnapshot(snapshotName, collectedBlocks, removedINodes);
     numSnapshots.getAndDecrement();
   }
 
   /**
    * Rename the given snapshot
-   * @param path
-   *          The directory path where the snapshot was taken
    * @param oldSnapshotName
    *          Old name of the snapshot
    * @param newSnapshotName
@@ -258,14 +250,11 @@ public class SnapshotManager implements SnapshotStatsMXBean {
    *           old name does not exist for the directory, and/or 3) there exists
    *           a snapshot with the new name for the directory
    */
-  public void renameSnapshot(final String path, final String oldSnapshotName,
-      final String newSnapshotName) throws IOException {
-    // Find the source root directory path where the snapshot was taken.
-    // All the check for path has been included in the valueOf method.
-    final INodeDirectory srcRoot = getSnapshottableRoot(path);
-    // Note that renameSnapshot and createSnapshot are synchronized externally
-    // through FSNamesystem's write lock
-    srcRoot.renameSnapshot(path, oldSnapshotName, newSnapshotName);
+  public void renameSnapshot(final INodesInPath iip, final String snapshotRoot,
+      final String oldSnapshotName, final String newSnapshotName)
+      throws IOException {
+    final INodeDirectory srcRoot = getSnapshottableRoot(iip);
+    srcRoot.renameSnapshot(snapshotRoot, oldSnapshotName, newSnapshotName);
   }
   
   public int getNumSnapshottableDirs() {
@@ -366,22 +355,23 @@ public class SnapshotManager implements SnapshotStatsMXBean {
    * Compute the difference between two snapshots of a directory, or between a
    * snapshot of the directory and its current tree.
    */
-  public SnapshotDiffReport diff(final String path, final String from,
+  public SnapshotDiffReport diff(final INodesInPath iip,
+      final String snapshotRootPath, final String from,
       final String to) throws IOException {
     // Find the source root directory path where the snapshots were taken.
     // All the check for path has been included in the valueOf method.
-    final INodeDirectory snapshotRoot = getSnapshottableRoot(path);
+    final INodeDirectory snapshotRoot = getSnapshottableRoot(iip);
 
     if ((from == null || from.isEmpty())
         && (to == null || to.isEmpty())) {
       // both fromSnapshot and toSnapshot indicate the current tree
-      return new SnapshotDiffReport(path, from, to,
+      return new SnapshotDiffReport(snapshotRootPath, from, to,
           Collections.<DiffReportEntry> emptyList());
     }
     final SnapshotDiffInfo diffs = snapshotRoot
         .getDirectorySnapshottableFeature().computeDiff(snapshotRoot, from, to);
     return diffs != null ? diffs.generateReport() : new SnapshotDiffReport(
-        path, from, to, Collections.<DiffReportEntry> emptyList());
+        snapshotRootPath, from, to, Collections.<DiffReportEntry> emptyList());
   }
   
   public void clearSnapshottableDirs() {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/475c6b49/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSPermissionChecker.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSPermissionChecker.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSPermissionChecker.java
index 7f23af4..883029a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSPermissionChecker.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSPermissionChecker.java
@@ -402,15 +402,17 @@ public class TestFSPermissionChecker {
 
   private void assertPermissionGranted(UserGroupInformation user, String path,
       FsAction access) throws IOException {
-    new FSPermissionChecker(SUPERUSER, SUPERGROUP, user).checkPermission(path,
-      dir, false, null, null, access, null, false, true);
+    INodesInPath iip = dir.getINodesInPath(path, true);
+    new FSPermissionChecker(SUPERUSER, SUPERGROUP, user).checkPermission(iip,
+      false, null, null, access, null, false);
   }
 
   private void assertPermissionDenied(UserGroupInformation user, String path,
       FsAction access) throws IOException {
     try {
-      new FSPermissionChecker(SUPERUSER, SUPERGROUP, user).checkPermission(path,
-        dir, false, null, null, access, null, false, true);
+      INodesInPath iip = dir.getINodesInPath(path, true);
+      new FSPermissionChecker(SUPERUSER, SUPERGROUP, user).checkPermission(iip,
+        false, null, null, access, null, false);
       fail("expected AccessControlException for user + " + user + ", path = " +
         path + ", access = " + access);
     } catch (AccessControlException e) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/475c6b49/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestSnapshotPathINodes.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestSnapshotPathINodes.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestSnapshotPathINodes.java
index 95524e8..d1a2377 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestSnapshotPathINodes.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestSnapshotPathINodes.java
@@ -48,22 +48,20 @@ public class TestSnapshotPathINodes {
   static private final Path file1 = new Path(sub1, "file1");
   static private final Path file2 = new Path(sub1, "file2");
 
-  static private Configuration conf;
   static private MiniDFSCluster cluster;
-  static private FSNamesystem fsn;
   static private FSDirectory fsdir;
 
   static private DistributedFileSystem hdfs;
 
   @BeforeClass
   public static void setUp() throws Exception {
-    conf = new Configuration();
+    Configuration conf = new Configuration();
     cluster = new MiniDFSCluster.Builder(conf)
       .numDataNodes(REPLICATION)
       .build();
     cluster.waitActive();
-    
-    fsn = cluster.getNamesystem();
+
+    FSNamesystem fsn = cluster.getNamesystem();
     fsdir = fsn.getFSDirectory();
     
     hdfs = cluster.getFileSystem();
@@ -136,7 +134,6 @@ public class TestSnapshotPathINodes {
   }
 
   /** 
-   * Test {@link INodeDirectory#getExistingPathINodes(byte[][], int, boolean)} 
    * for normal (non-snapshot) file.
    */
   @Test (timeout=15000)
@@ -180,7 +177,6 @@ public class TestSnapshotPathINodes {
   }
   
   /** 
-   * Test {@link INodeDirectory#getExistingPathINodes(byte[][], int, boolean)} 
    * for snapshot file.
    */
   @Test (timeout=15000)
@@ -259,7 +255,6 @@ public class TestSnapshotPathINodes {
   }
   
   /** 
-   * Test {@link INodeDirectory#getExistingPathINodes(byte[][], int, boolean)} 
    * for snapshot file after deleting the original file.
    */
   @Test (timeout=15000)
@@ -316,11 +311,8 @@ public class TestSnapshotPathINodes {
     hdfs.deleteSnapshot(sub1, "s2");
     hdfs.disallowSnapshot(sub1);
   }
-  
-  static private Snapshot s4;
 
-  /** 
-   * Test {@link INodeDirectory#getExistingPathINodes(byte[][], int, boolean)} 
+  /**
    * for snapshot file while adding a new file after snapshot.
    */
   @Test (timeout=15000)
@@ -333,7 +325,8 @@ public class TestSnapshotPathINodes {
     // Add a new file /TestSnapshot/sub1/file3
     final Path file3 = new Path(sub1, "file3");
     DFSTestUtil.createFile(hdfs, file3, 1024, REPLICATION, seed);
-  
+
+    Snapshot s4;
     {
       // Check the inodes for /TestSnapshot/sub1/.snapshot/s4/file3
       String snapshotPath = sub1.toString() + "/.snapshot/s4/file3";
@@ -379,7 +372,6 @@ public class TestSnapshotPathINodes {
   }
   
   /** 
-   * Test {@link INodeDirectory#getExistingPathINodes(byte[][], int, boolean)} 
    * for snapshot file while modifying file after snapshot.
    */
   @Test (timeout=15000)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/475c6b49/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSnapshotManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSnapshotManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSnapshotManager.java
index 617bec8..ac6acf9 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSnapshotManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSnapshotManager.java
@@ -18,6 +18,7 @@
 
 package org.apache.hadoop.hdfs.server.namenode.snapshot;
 
+import static org.mockito.Matchers.anyObject;
 import static org.mockito.Matchers.anyString;
 import static org.mockito.Mockito.doReturn;
 import static org.mockito.Mockito.mock;
@@ -29,6 +30,7 @@ import org.apache.hadoop.hdfs.protocol.SnapshotException;
 import org.apache.hadoop.hdfs.server.namenode.FSDirectory;
 import org.apache.hadoop.hdfs.server.namenode.INode;
 import org.apache.hadoop.hdfs.server.namenode.INodeDirectory;
+import org.apache.hadoop.hdfs.server.namenode.INodesInPath;
 import org.junit.Assert;
 import org.junit.Test;
 
@@ -48,22 +50,23 @@ public class TestSnapshotManager {
     //
     INodeDirectory ids = mock(INodeDirectory.class);
     FSDirectory fsdir = mock(FSDirectory.class);
+    INodesInPath iip = mock(INodesInPath.class);
 
     SnapshotManager sm = spy(new SnapshotManager(fsdir));
-    doReturn(ids).when(sm).getSnapshottableRoot(anyString());
+    doReturn(ids).when(sm).getSnapshottableRoot((INodesInPath) anyObject());
     doReturn(testMaxSnapshotLimit).when(sm).getMaxSnapshotID();
 
     // Create testMaxSnapshotLimit snapshots. These should all succeed.
     //
     for (Integer i = 0; i < testMaxSnapshotLimit; ++i) {
-      sm.createSnapshot("dummy", i.toString());
+      sm.createSnapshot(iip, "dummy", i.toString());
     }
 
     // Attempt to create one more snapshot. This should fail due to snapshot
     // ID rollover.
     //
     try {
-      sm.createSnapshot("dummy", "shouldFailSnapshot");
+      sm.createSnapshot(iip, "dummy", "shouldFailSnapshot");
       Assert.fail("Expected SnapshotException not thrown");
     } catch (SnapshotException se) {
       Assert.assertTrue(
@@ -72,13 +75,14 @@ public class TestSnapshotManager {
 
     // Delete a snapshot to free up a slot.
     //
-    sm.deleteSnapshot("", "", mock(INode.BlocksMapUpdateInfo.class), new ArrayList<INode>());
+    sm.deleteSnapshot(iip, "", mock(INode.BlocksMapUpdateInfo.class),
+        new ArrayList<INode>());
 
     // Attempt to create a snapshot again. It should still fail due
     // to snapshot ID rollover.
     //
     try {
-      sm.createSnapshot("dummy", "shouldFailSnapshot2");
+      sm.createSnapshot(iip, "dummy", "shouldFailSnapshot2");
       Assert.fail("Expected SnapshotException not thrown");
     } catch (SnapshotException se) {
       Assert.assertTrue(


[10/29] hadoop git commit: HDFS-7424. Add web UI for NFS gateway. Contributed by Brandon Li

Posted by vi...@apache.org.
HDFS-7424. Add web UI for NFS gateway. Contributed by Brandon Li


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

Branch: refs/heads/HDFS-EC
Commit: 1bbcc3d0320b9435317bfeaa078af22d4de8d00c
Parents: 9d1a8f5
Author: Brandon Li <br...@apache.org>
Authored: Thu Dec 4 10:46:26 2014 -0800
Committer: Brandon Li <br...@apache.org>
Committed: Thu Dec 4 10:46:26 2014 -0800

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs-nfs/pom.xml     |   5 +
 .../hadoop/hdfs/nfs/conf/NfsConfigKeys.java     |  10 ++
 .../hadoop/hdfs/nfs/nfs3/Nfs3HttpServer.java    | 111 +++++++++++++++++++
 .../hadoop/hdfs/nfs/nfs3/RpcProgramNfs3.java    |  24 +++-
 .../hdfs/nfs/nfs3/TestNfs3HttpServer.java       |  89 +++++++++++++++
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |   2 +
 hadoop-hdfs-project/hadoop-hdfs/pom.xml         |   3 +
 7 files changed, 242 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/1bbcc3d0/hadoop-hdfs-project/hadoop-hdfs-nfs/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-nfs/pom.xml b/hadoop-hdfs-project/hadoop-hdfs-nfs/pom.xml
index 42962a6..9a9d29c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-nfs/pom.xml
+++ b/hadoop-hdfs-project/hadoop-hdfs-nfs/pom.xml
@@ -179,6 +179,11 @@ http://maven.apache.org/xsd/maven-4.0.0.xsd">
       <artifactId>xmlenc</artifactId>
       <scope>compile</scope>
     </dependency>
+    <dependency>
+      <groupId>org.bouncycastle</groupId>
+      <artifactId>bcprov-jdk16</artifactId>
+      <scope>test</scope>
+    </dependency>
   </dependencies>
 
   <profiles>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1bbcc3d0/hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/conf/NfsConfigKeys.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/conf/NfsConfigKeys.java b/hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/conf/NfsConfigKeys.java
index 178d855..7566791 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/conf/NfsConfigKeys.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/conf/NfsConfigKeys.java
@@ -60,4 +60,14 @@ public class NfsConfigKeys {
   
   public final static String LARGE_FILE_UPLOAD = "nfs.large.file.upload";
   public final static boolean LARGE_FILE_UPLOAD_DEFAULT = true;
+  
+  public static final String NFS_HTTP_PORT_KEY = "nfs.http.port";
+  public static final int NFS_HTTP_PORT_DEFAULT = 50079;
+  public static final String NFS_HTTP_ADDRESS_KEY = "nfs.http.address";
+  public static final String NFS_HTTP_ADDRESS_DEFAULT = "0.0.0.0:" + NFS_HTTP_PORT_DEFAULT;
+
+  public static final String NFS_HTTPS_PORT_KEY = "nfs.https.port";
+  public static final int NFS_HTTPS_PORT_DEFAULT = 50579;
+  public static final String NFS_HTTPS_ADDRESS_KEY = "nfs.https.address";
+  public static final String NFS_HTTPS_ADDRESS_DEFAULT = "0.0.0.0:" + NFS_HTTPS_PORT_DEFAULT;
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1bbcc3d0/hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/Nfs3HttpServer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/Nfs3HttpServer.java b/hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/Nfs3HttpServer.java
new file mode 100644
index 0000000..c37a21e
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/Nfs3HttpServer.java
@@ -0,0 +1,111 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.nfs.nfs3;
+
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.net.URI;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.DFSUtil;
+import org.apache.hadoop.hdfs.nfs.conf.NfsConfigKeys;
+import org.apache.hadoop.hdfs.nfs.conf.NfsConfiguration;
+import org.apache.hadoop.hdfs.server.common.JspHelper;
+import org.apache.hadoop.http.HttpConfig;
+import org.apache.hadoop.http.HttpServer2;
+import org.apache.hadoop.net.NetUtils;
+
+/**
+ * Encapsulates the HTTP server started by the NFS3 gateway.
+ */
+class Nfs3HttpServer {
+  private int infoPort;
+  private int infoSecurePort;
+
+  private HttpServer2 httpServer;
+
+  private final NfsConfiguration conf;
+
+  Nfs3HttpServer(NfsConfiguration conf) {
+    this.conf = conf;
+  }
+
+  void start() throws IOException {
+    final InetSocketAddress httpAddr = getHttpAddress(conf);
+
+    final String httpsAddrString = conf.get(
+        NfsConfigKeys.NFS_HTTPS_ADDRESS_KEY,
+        NfsConfigKeys.NFS_HTTPS_ADDRESS_DEFAULT);
+    InetSocketAddress httpsAddr = NetUtils.createSocketAddr(httpsAddrString);
+
+    HttpServer2.Builder builder = DFSUtil.httpServerTemplateForNNAndJN(conf,
+        httpAddr, httpsAddr, "nfs3",
+        NfsConfigKeys.DFS_NFS_KERBEROS_PRINCIPAL_KEY,
+        NfsConfigKeys.DFS_NFS_KEYTAB_FILE_KEY);
+
+    this.httpServer = builder.build();
+    this.httpServer.start();
+    
+    HttpConfig.Policy policy = DFSUtil.getHttpPolicy(conf);
+    int connIdx = 0;
+    if (policy.isHttpEnabled()) {
+      infoPort = httpServer.getConnectorAddress(connIdx++).getPort();
+    }
+
+    if (policy.isHttpsEnabled()) {
+      infoSecurePort = httpServer.getConnectorAddress(connIdx).getPort();
+    }
+  }
+
+  void stop() throws IOException {
+    if (httpServer != null) {
+      try {
+        httpServer.stop();
+      } catch (Exception e) {
+        throw new IOException(e);
+      }
+    }
+  }
+
+  public int getPort() {
+    return this.infoPort;
+  }
+
+  public int getSecurePort() {
+    return this.infoSecurePort;
+  }
+
+  /**
+   * Return the URI that locates the HTTP server.
+   */
+  public URI getServerURI() {
+    // getHttpClientScheme() only returns https for HTTPS_ONLY policy. This
+    // matches the behavior that the first connector is a HTTPS connector only
+    // for HTTPS_ONLY policy.
+    InetSocketAddress addr = httpServer.getConnectorAddress(0);
+    return URI.create(DFSUtil.getHttpClientScheme(conf) + "://"
+        + NetUtils.getHostPortString(addr));
+  }
+
+  public InetSocketAddress getHttpAddress(Configuration conf) {
+    String addr = conf.get(NfsConfigKeys.NFS_HTTP_ADDRESS_KEY,
+        NfsConfigKeys.NFS_HTTP_ADDRESS_DEFAULT);
+    return NetUtils.createSocketAddr(addr, NfsConfigKeys.NFS_HTTP_PORT_DEFAULT,
+        NfsConfigKeys.NFS_HTTP_ADDRESS_KEY);
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1bbcc3d0/hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/RpcProgramNfs3.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/RpcProgramNfs3.java b/hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/RpcProgramNfs3.java
index f86dbec..c860dd5 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/RpcProgramNfs3.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/RpcProgramNfs3.java
@@ -162,6 +162,7 @@ public class RpcProgramNfs3 extends RpcProgram implements Nfs3Interface {
 
   private final RpcCallCache rpcCallCache;
   private JvmPauseMonitor pauseMonitor;
+  private Nfs3HttpServer infoServer = null;
 
   public RpcProgramNfs3(NfsConfiguration config, DatagramSocket registrationSocket,
       boolean allowInsecurePorts) throws IOException {
@@ -204,6 +205,7 @@ public class RpcProgramNfs3 extends RpcProgram implements Nfs3Interface {
     }
 
     rpcCallCache = new RpcCallCache("NFS3", 256);
+    infoServer = new Nfs3HttpServer(config);
   }
 
   private void clearDirectory(String writeDumpDir) throws IOException {
@@ -220,14 +222,19 @@ public class RpcProgramNfs3 extends RpcProgram implements Nfs3Interface {
       throw new IOException("Cannot create dump directory " + dumpDir);
     }
   }
-
+  
   @Override
-  public void startDaemons() {
+  public void startDaemons() {    
     if (pauseMonitor == null) {
       pauseMonitor = new JvmPauseMonitor(config);
       pauseMonitor.start();
     }
     writeManager.startAsyncDataSerivce();
+    try {
+      infoServer.start();
+    } catch (IOException e) {
+      LOG.error("failed to start web server", e);
+    }
   }
 
   @Override
@@ -238,6 +245,19 @@ public class RpcProgramNfs3 extends RpcProgram implements Nfs3Interface {
     if (pauseMonitor != null) {
       pauseMonitor.stop();
     }
+    // Stop the web server
+    if (infoServer != null) {
+      try {
+        infoServer.stop();
+      } catch (Exception e) {
+        LOG.warn("Exception shutting down web server", e);
+      }
+    }
+  }
+  
+  @VisibleForTesting
+  Nfs3HttpServer getInfoServer() {
+    return this.infoServer;
   }
 
   // Checks the type of IOException and maps it to appropriate Nfs3Status code.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1bbcc3d0/hadoop-hdfs-project/hadoop-hdfs-nfs/src/test/java/org/apache/hadoop/hdfs/nfs/nfs3/TestNfs3HttpServer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-nfs/src/test/java/org/apache/hadoop/hdfs/nfs/nfs3/TestNfs3HttpServer.java b/hadoop-hdfs-project/hadoop-hdfs-nfs/src/test/java/org/apache/hadoop/hdfs/nfs/nfs3/TestNfs3HttpServer.java
new file mode 100644
index 0000000..d44e9ab
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-nfs/src/test/java/org/apache/hadoop/hdfs/nfs/nfs3/TestNfs3HttpServer.java
@@ -0,0 +1,89 @@
+/**
+ * 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.nfs.nfs3;
+
+import static org.junit.Assert.assertTrue;
+
+import java.io.File;
+import java.net.URL;
+
+import org.apache.hadoop.fs.FileUtil;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.DFSTestUtil;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.nfs.conf.NfsConfigKeys;
+import org.apache.hadoop.hdfs.nfs.conf.NfsConfiguration;
+import org.apache.hadoop.http.HttpConfig;
+import org.apache.hadoop.security.ssl.KeyStoreTestUtil;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+public class TestNfs3HttpServer {
+  private static final String BASEDIR = System.getProperty("test.build.dir",
+      "target/test-dir") + "/" + TestNfs3HttpServer.class.getSimpleName();
+  private static NfsConfiguration conf = new NfsConfiguration();
+  private static MiniDFSCluster cluster;
+  private static String keystoresDir;
+  private static String sslConfDir;
+
+  @BeforeClass
+  public static void setUp() throws Exception {
+    conf.set(DFSConfigKeys.DFS_HTTP_POLICY_KEY,
+        HttpConfig.Policy.HTTP_AND_HTTPS.name());
+    conf.set(NfsConfigKeys.NFS_HTTP_ADDRESS_KEY, "localhost:0");
+    conf.set(NfsConfigKeys.NFS_HTTPS_ADDRESS_KEY, "localhost:0");
+    File base = new File(BASEDIR);
+    FileUtil.fullyDelete(base);
+    base.mkdirs();
+    keystoresDir = new File(BASEDIR).getAbsolutePath();
+    sslConfDir = KeyStoreTestUtil.getClasspathDir(TestNfs3HttpServer.class);
+    KeyStoreTestUtil.setupSSLConfig(keystoresDir, sslConfDir, conf, false);
+
+    cluster = new MiniDFSCluster.Builder(conf).numDataNodes(1).build();
+    cluster.waitActive();
+  }
+
+  @AfterClass
+  public static void tearDown() throws Exception {
+    FileUtil.fullyDelete(new File(BASEDIR));
+    if (cluster != null) {
+      cluster.shutdown();
+    }
+    KeyStoreTestUtil.cleanupSSLConfig(keystoresDir, sslConfDir);
+  }
+
+  @Test
+  public void testHttpServer() throws Exception {
+    Nfs3 nfs = new Nfs3(conf);
+    nfs.startServiceInternal(false);
+    RpcProgramNfs3 nfsd = (RpcProgramNfs3) nfs.getRpcProgram();
+    Nfs3HttpServer infoServer = nfsd.getInfoServer();
+
+    String urlRoot = infoServer.getServerURI().toString();
+
+    // Check default servlets.
+    String pageContents = DFSTestUtil.urlGet(new URL(urlRoot + "/jmx"));
+    assertTrue("Bad contents: " + pageContents,
+        pageContents.contains("java.lang:type="));
+    System.out.println("pc:" + pageContents);
+
+    int port = infoServer.getSecurePort();
+    assertTrue("Can't get https port", port > 0);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1bbcc3d0/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index a244dab..2775285 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -272,6 +272,8 @@ Release 2.7.0 - UNRELEASED
     HDFS-6982. nntop: top­-like tool for name node users.
     (Maysam Yabandeh via wang)
 
+    HDFS-7424. Add web UI for NFS gateway (brandonli)
+
   IMPROVEMENTS
 
     HDFS-7055. Add tracing to DFSInputStream (cmccabe)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1bbcc3d0/hadoop-hdfs-project/hadoop-hdfs/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/pom.xml b/hadoop-hdfs-project/hadoop-hdfs/pom.xml
index 2aab073..bad1792 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/pom.xml
+++ b/hadoop-hdfs-project/hadoop-hdfs/pom.xml
@@ -252,6 +252,9 @@ http://maven.apache.org/xsd/maven-4.0.0.xsd">
                 <copy file="${basedir}/src/main/webapps/proto-web.xml"
                       tofile="${project.build.directory}/webapps/journal/WEB-INF/web.xml"
                       filtering="true"/>
+                <copy file="${basedir}/src/main/webapps/proto-web.xml"
+                      tofile="${project.build.directory}/webapps/nfs3/WEB-INF/web.xml"
+                      filtering="true"/>
                 <copy toDir="${project.build.directory}/webapps">
                   <fileset dir="${basedir}/src/main/webapps">
                     <exclude name="**/proto-web.xml"/>


[19/29] hadoop git commit: HADOOP-11355. When accessing data in HDFS and the key has been deleted, a Null Pointer Exception is shown. Contributed by Arun Suresh.

Posted by vi...@apache.org.
HADOOP-11355. When accessing data in HDFS and the key has been deleted, a Null Pointer Exception is shown. Contributed by Arun Suresh.


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

Branch: refs/heads/HDFS-EC
Commit: 9cdaec6a6f6cb1680ad6e44d7b0c8d70cdcbe3fa
Parents: f6452eb
Author: Andrew Wang <wa...@apache.org>
Authored: Fri Dec 5 12:01:23 2014 -0800
Committer: Andrew Wang <wa...@apache.org>
Committed: Fri Dec 5 12:01:23 2014 -0800

----------------------------------------------------------------------
 hadoop-common-project/hadoop-common/CHANGES.txt              | 3 +++
 .../crypto/key/kms/server/KeyAuthorizationKeyProvider.java   | 4 ++++
 .../org/apache/hadoop/crypto/key/kms/server/TestKMS.java     | 8 ++++++++
 3 files changed, 15 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/9cdaec6a/hadoop-common-project/hadoop-common/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/CHANGES.txt b/hadoop-common-project/hadoop-common/CHANGES.txt
index 2f88fc8..7a6a938 100644
--- a/hadoop-common-project/hadoop-common/CHANGES.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES.txt
@@ -505,6 +505,9 @@ Release 2.7.0 - UNRELEASED
     HADOOP-11348. Remove unused variable from CMake error message for finding
     openssl (Dian Fu via Colin P. McCabe)
 
+    HADOOP-11355. When accessing data in HDFS and the key has been deleted,
+    a Null Pointer Exception is shown. (Arun Suresh via wang)
+
 Release 2.6.0 - 2014-11-18
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9cdaec6a/hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KeyAuthorizationKeyProvider.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KeyAuthorizationKeyProvider.java b/hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KeyAuthorizationKeyProvider.java
index 4ce9611..074f1fb 100644
--- a/hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KeyAuthorizationKeyProvider.java
+++ b/hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KeyAuthorizationKeyProvider.java
@@ -240,6 +240,10 @@ public class KeyAuthorizationKeyProvider extends KeyProviderCryptoExtension {
     String kn = ekv.getEncryptionKeyName();
     String kvn = ekv.getEncryptionKeyVersionName();
     KeyVersion kv = provider.getKeyVersion(kvn);
+    if (kv == null) {
+      throw new IllegalArgumentException(String.format(
+          "'%s' not found", kvn));
+    }
     if (!kv.getName().equals(kn)) {
       throw new IllegalArgumentException(String.format(
           "KeyVersion '%s' does not belong to the key '%s'", kvn, kn));

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9cdaec6a/hadoop-common-project/hadoop-kms/src/test/java/org/apache/hadoop/crypto/key/kms/server/TestKMS.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-kms/src/test/java/org/apache/hadoop/crypto/key/kms/server/TestKMS.java b/hadoop-common-project/hadoop-kms/src/test/java/org/apache/hadoop/crypto/key/kms/server/TestKMS.java
index b9409ca..61ce807 100644
--- a/hadoop-common-project/hadoop-kms/src/test/java/org/apache/hadoop/crypto/key/kms/server/TestKMS.java
+++ b/hadoop-common-project/hadoop-kms/src/test/java/org/apache/hadoop/crypto/key/kms/server/TestKMS.java
@@ -498,6 +498,14 @@ public class TestKMS {
         // deleteKey()
         kp.deleteKey("k1");
 
+        // Check decryption after Key deletion
+        try {
+          kpExt.decryptEncryptedKey(ek1);
+          Assert.fail("Should not be allowed !!");
+        } catch (Exception e) {
+          Assert.assertTrue(e.getMessage().contains("'k1@1' not found"));
+        }
+
         // getKey()
         Assert.assertNull(kp.getKeyVersion("k1"));
 


[13/29] hadoop git commit: HDFS-7468. Moving verify* functions to corresponding classes. Contributed by Li Lu.

Posted by vi...@apache.org.
HDFS-7468. Moving verify* functions to corresponding classes. Contributed by Li Lu.


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

Branch: refs/heads/HDFS-EC
Commit: 26d8dec756da1d9bd3df3b41a4dd5d8ff03bc5b2
Parents: 258623f
Author: Haohui Mai <wh...@apache.org>
Authored: Thu Dec 4 14:09:45 2014 -0800
Committer: Haohui Mai <wh...@apache.org>
Committed: Thu Dec 4 14:09:45 2014 -0800

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |  3 +
 .../hdfs/server/namenode/FSDirRenameOp.java     | 54 +++++++++++++--
 .../hdfs/server/namenode/FSDirSnapshotOp.java   | 20 +++++-
 .../hdfs/server/namenode/FSDirectory.java       | 72 ++------------------
 4 files changed, 78 insertions(+), 71 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/26d8dec7/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index 2775285..4432024 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -427,6 +427,9 @@ Release 2.7.0 - UNRELEASED
     HDFS-7458. Add description to the nfs ports in core-site.xml used by nfs
     test to avoid confusion (Yongjun Zhang via brandonli)
 
+    HDFS-7468. Moving verify* functions to corresponding classes.
+    (Li Lu via wheat9)
+
   OPTIMIZATIONS
 
   BUG FIXES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/26d8dec7/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirRenameOp.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirRenameOp.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirRenameOp.java
index f371f05..08241c4 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirRenameOp.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirRenameOp.java
@@ -25,6 +25,7 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.permission.FsAction;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
+import org.apache.hadoop.hdfs.protocol.FSLimitException;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
 import org.apache.hadoop.hdfs.protocol.SnapshotException;
@@ -73,6 +74,51 @@ class FSDirRenameOp {
   }
 
   /**
+   * Verify quota for rename operation where srcInodes[srcInodes.length-1] moves
+   * dstInodes[dstInodes.length-1]
+   */
+  static void verifyQuotaForRename(FSDirectory fsd,
+      INode[] src, INode[] dst)
+      throws QuotaExceededException {
+    if (!fsd.getFSNamesystem().isImageLoaded() || fsd.shouldSkipQuotaChecks()) {
+      // Do not check quota if edits log is still being processed
+      return;
+    }
+    int i = 0;
+    while(src[i] == dst[i]) { i++; }
+    // src[i - 1] is the last common ancestor.
+
+    final Quota.Counts delta = src[src.length - 1].computeQuotaUsage();
+
+    // Reduce the required quota by dst that is being removed
+    final int dstIndex = dst.length - 1;
+    if (dst[dstIndex] != null) {
+      delta.subtract(dst[dstIndex].computeQuotaUsage());
+    }
+    FSDirectory.verifyQuota(dst, dstIndex, delta.get(Quota.NAMESPACE),
+        delta.get(Quota.DISKSPACE), src[i - 1]);
+  }
+
+  /**
+   * Checks file system limits (max component length and max directory items)
+   * during a rename operation.
+   */
+  static void verifyFsLimitsForRename(FSDirectory fsd,
+      INodesInPath srcIIP,
+      INodesInPath dstIIP)
+      throws FSLimitException.PathComponentTooLongException,
+          FSLimitException.MaxDirectoryItemsExceededException {
+    byte[] dstChildName = dstIIP.getLastLocalName();
+    INode[] dstInodes = dstIIP.getINodes();
+    int pos = dstInodes.length - 1;
+    fsd.verifyMaxComponentLength(dstChildName, dstInodes, pos);
+    // Do not enforce max directory items if renaming within same directory.
+    if (srcIIP.getINode(-2) != dstIIP.getINode(-2)) {
+      fsd.verifyMaxDirItems(dstInodes, pos);
+    }
+  }
+
+  /**
    * Change a path name
    *
    * @param fsd FSDirectory
@@ -129,8 +175,8 @@ class FSDirRenameOp {
 
     fsd.ezManager.checkMoveValidity(srcIIP, dstIIP, src);
     // Ensure dst has quota to accommodate rename
-    fsd.verifyFsLimitsForRename(srcIIP, dstIIP);
-    fsd.verifyQuotaForRename(srcIIP.getINodes(), dstIIP.getINodes());
+    verifyFsLimitsForRename(fsd, srcIIP, dstIIP);
+    verifyQuotaForRename(fsd, srcIIP.getINodes(), dstIIP.getINodes());
 
     RenameOperation tx = new RenameOperation(fsd, src, dst, srcIIP, dstIIP);
 
@@ -310,8 +356,8 @@ class FSDirRenameOp {
     }
 
     // Ensure dst has quota to accommodate rename
-    fsd.verifyFsLimitsForRename(srcIIP, dstIIP);
-    fsd.verifyQuotaForRename(srcIIP.getINodes(), dstIIP.getINodes());
+    verifyFsLimitsForRename(fsd, srcIIP, dstIIP);
+    verifyQuotaForRename(fsd, srcIIP.getINodes(), dstIIP.getINodes());
 
     RenameOperation tx = new RenameOperation(fsd, src, dst, srcIIP, dstIIP);
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/26d8dec7/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirSnapshotOp.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirSnapshotOp.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirSnapshotOp.java
index aa751a7..bfd7019 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirSnapshotOp.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirSnapshotOp.java
@@ -17,9 +17,12 @@
  */
 package org.apache.hadoop.hdfs.server.namenode;
 
+import org.apache.hadoop.HadoopIllegalArgumentException;
 import org.apache.hadoop.fs.InvalidPathException;
+import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.permission.FsAction;
 import org.apache.hadoop.hdfs.DFSUtil;
+import org.apache.hadoop.hdfs.protocol.FSLimitException;
 import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport;
 import org.apache.hadoop.hdfs.protocol.SnapshotException;
 import org.apache.hadoop.hdfs.protocol.SnapshottableDirectoryStatus;
@@ -32,6 +35,19 @@ import java.io.IOException;
 import java.util.List;
 
 class FSDirSnapshotOp {
+  /** Verify if the snapshot name is legal. */
+  static void verifySnapshotName(FSDirectory fsd, String snapshotName,
+      String path)
+      throws FSLimitException.PathComponentTooLongException {
+    if (snapshotName.contains(Path.SEPARATOR)) {
+      throw new HadoopIllegalArgumentException(
+          "Snapshot name cannot contain \"" + Path.SEPARATOR + "\"");
+    }
+    final byte[] bytes = DFSUtil.string2Bytes(snapshotName);
+    fsd.verifyINodeName(bytes);
+    fsd.verifyMaxComponentLength(bytes, path, 0);
+  }
+
   /** Allow snapshot on a directory. */
   static void allowSnapshot(FSDirectory fsd, SnapshotManager snapshotManager,
                             String path) throws IOException {
@@ -82,7 +98,7 @@ class FSDirSnapshotOp {
             snapshotName);
       }
     }
-    fsd.verifySnapshotName(snapshotName, snapshotRoot);
+    verifySnapshotName(fsd, snapshotName, snapshotRoot);
     fsd.writeLock();
     try {
       snapshotPath = snapshotManager.createSnapshot(snapshotRoot, snapshotName);
@@ -103,7 +119,7 @@ class FSDirSnapshotOp {
       FSPermissionChecker pc = fsd.getPermissionChecker();
         fsd.checkOwner(pc, path);
     }
-    fsd.verifySnapshotName(snapshotNewName, path);
+    verifySnapshotName(fsd, snapshotNewName, path);
     fsd.writeLock();
     try {
       snapshotManager.renameSnapshot(path, snapshotOldName, snapshotNewName);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/26d8dec7/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 ffc2653..950c9fe 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
@@ -291,6 +291,10 @@ public class FSDirectory implements Closeable {
     }
   }
 
+  boolean shouldSkipQuotaChecks() {
+    return skipQuotaCheck;
+  }
+
   /** Enable quota verification */
   void enableQuotaChecks() {
     skipQuotaCheck = false;
@@ -1095,7 +1099,7 @@ public class FSDirectory implements Closeable {
    *          Pass null if a node is not being moved.
    * @throws QuotaExceededException if quota limit is exceeded.
    */
-  private static void verifyQuota(INode[] inodes, int pos, long nsDelta,
+  static void verifyQuota(INode[] inodes, int pos, long nsDelta,
       long dsDelta, INode commonAncestor) throws QuotaExceededException {
     if (nsDelta <= 0 && dsDelta <= 0) {
       // if quota is being freed or not being consumed
@@ -1120,69 +1124,7 @@ public class FSDirectory implements Closeable {
       }
     }
   }
-  
-  /**
-   * Verify quota for rename operation where srcInodes[srcInodes.length-1] moves
-   * dstInodes[dstInodes.length-1]
-   * 
-   * @param src directory from where node is being moved.
-   * @param dst directory to where node is moved to.
-   * @throws QuotaExceededException if quota limit is exceeded.
-   */
-  void verifyQuotaForRename(INode[] src, INode[] dst)
-      throws QuotaExceededException {
-    if (!namesystem.isImageLoaded() || skipQuotaCheck) {
-      // Do not check quota if edits log is still being processed
-      return;
-    }
-    int i = 0;
-    while(src[i] == dst[i]) { i++; }
-    // src[i - 1] is the last common ancestor.
 
-    final Quota.Counts delta = src[src.length - 1].computeQuotaUsage();
-    
-    // Reduce the required quota by dst that is being removed
-    final int dstIndex = dst.length - 1;
-    if (dst[dstIndex] != null) {
-      delta.subtract(dst[dstIndex].computeQuotaUsage());
-    }
-    verifyQuota(dst, dstIndex, delta.get(Quota.NAMESPACE),
-        delta.get(Quota.DISKSPACE), src[i - 1]);
-  }
-
-  /**
-   * Checks file system limits (max component length and max directory items)
-   * during a rename operation.
-   *
-   * @param srcIIP INodesInPath containing every inode in the rename source
-   * @param dstIIP INodesInPath containing every inode in the rename destination
-   * @throws PathComponentTooLongException child's name is too long.
-   * @throws MaxDirectoryItemsExceededException too many children.
-   */
-  void verifyFsLimitsForRename(INodesInPath srcIIP, INodesInPath dstIIP)
-      throws PathComponentTooLongException, MaxDirectoryItemsExceededException {
-    byte[] dstChildName = dstIIP.getLastLocalName();
-    INode[] dstInodes = dstIIP.getINodes();
-    int pos = dstInodes.length - 1;
-    verifyMaxComponentLength(dstChildName, dstInodes, pos);
-    // Do not enforce max directory items if renaming within same directory.
-    if (srcIIP.getINode(-2) != dstIIP.getINode(-2)) {
-      verifyMaxDirItems(dstInodes, pos);
-    }
-  }
-
-  /** Verify if the snapshot name is legal. */
-  void verifySnapshotName(String snapshotName, String path)
-      throws PathComponentTooLongException {
-    if (snapshotName.contains(Path.SEPARATOR)) {
-      throw new HadoopIllegalArgumentException(
-          "Snapshot name cannot contain \"" + Path.SEPARATOR + "\"");
-    }
-    final byte[] bytes = DFSUtil.string2Bytes(snapshotName);
-    verifyINodeName(bytes);
-    verifyMaxComponentLength(bytes, path, 0);
-  }
-  
   /** Verify if the inode name is legal. */
   void verifyINodeName(byte[] childName) throws HadoopIllegalArgumentException {
     if (Arrays.equals(HdfsConstants.DOT_SNAPSHOT_DIR_BYTES, childName)) {
@@ -1202,7 +1144,7 @@ public class FSDirectory implements Closeable {
    * @param pos int position of new child in path
    * @throws PathComponentTooLongException child's name is too long.
    */
-  private void verifyMaxComponentLength(byte[] childName, Object parentPath,
+  void verifyMaxComponentLength(byte[] childName, Object parentPath,
       int pos) throws PathComponentTooLongException {
     if (maxComponentLength == 0) {
       return;
@@ -1230,7 +1172,7 @@ public class FSDirectory implements Closeable {
    * @param pos int position of new child in pathComponents
    * @throws MaxDirectoryItemsExceededException too many children.
    */
-  private void verifyMaxDirItems(INode[] pathComponents, int pos)
+  void verifyMaxDirItems(INode[] pathComponents, int pos)
       throws MaxDirectoryItemsExceededException {
 
     final INodeDirectory parent = pathComponents[pos-1].asDirectory();


[28/29] hadoop git commit: YARN-2927. [YARN-1492] InMemorySCMStore properties are inconsistent. (Ray Chiang via kasha)

Posted by vi...@apache.org.
YARN-2927. [YARN-1492] InMemorySCMStore properties are inconsistent. (Ray Chiang via kasha)


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

Branch: refs/heads/HDFS-EC
Commit: 120e1decd7f6861e753269690d454cb14c240857
Parents: 1b3bb9e
Author: Karthik Kambatla <ka...@apache.org>
Authored: Sun Dec 7 22:28:26 2014 -0800
Committer: Karthik Kambatla <ka...@apache.org>
Committed: Sun Dec 7 22:28:26 2014 -0800

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                                   | 3 +++
 .../main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java  | 2 +-
 2 files changed, 4 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/120e1dec/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index 0d7a843..43b19ec 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -197,6 +197,9 @@ Release 2.7.0 - UNRELEASED
     YARN-2869. CapacityScheduler should trim sub queue names when parse
     configuration. (Wangda Tan via jianhe)
 
+    YARN-2927. [YARN-1492] InMemorySCMStore properties are inconsistent. 
+    (Ray Chiang via kasha)
+
 Release 2.6.0 - 2014-11-18
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/120e1dec/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 10ba832..55073c5 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
@@ -1416,7 +1416,7 @@ public class YarnConfiguration extends Configuration {
   // In-memory SCM store configuration
   
   public static final String IN_MEMORY_STORE_PREFIX =
-      SHARED_CACHE_PREFIX + "in-memory.";
+      SCM_STORE_PREFIX + "in-memory.";
 
   /**
    * A resource in the InMemorySCMStore is considered stale if the time since


[27/29] hadoop git commit: HADOOP-11313. Adding a document about NativeLibraryChecker. Contributed by Tsuyoshi OZAWA.

Posted by vi...@apache.org.
HADOOP-11313. Adding a document about NativeLibraryChecker. Contributed by Tsuyoshi OZAWA.


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

Branch: refs/heads/HDFS-EC
Commit: 1b3bb9e7a33716c4d94786598b91a24a4b29fe67
Parents: 9297f98
Author: cnauroth <cn...@apache.org>
Authored: Sat Dec 6 20:12:31 2014 -0800
Committer: cnauroth <cn...@apache.org>
Committed: Sat Dec 6 20:12:31 2014 -0800

----------------------------------------------------------------------
 hadoop-common-project/hadoop-common/CHANGES.txt   |  3 +++
 .../src/site/apt/NativeLibraries.apt.vm           | 18 ++++++++++++++++++
 2 files changed, 21 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/1b3bb9e7/hadoop-common-project/hadoop-common/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/CHANGES.txt b/hadoop-common-project/hadoop-common/CHANGES.txt
index 965c6d3..a626388 100644
--- a/hadoop-common-project/hadoop-common/CHANGES.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES.txt
@@ -413,6 +413,9 @@ Release 2.7.0 - UNRELEASED
     HADOOP-11356. Removed deprecated o.a.h.fs.permission.AccessControlException.
     (Li Lu via wheat9)
 
+    HADOOP-11313. Adding a document about NativeLibraryChecker.
+    (Tsuyoshi OZAWA via cnauroth)
+
   OPTIMIZATIONS
 
     HADOOP-11323. WritableComparator#compare keeps reference to byte array.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1b3bb9e7/hadoop-common-project/hadoop-common/src/site/apt/NativeLibraries.apt.vm
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/site/apt/NativeLibraries.apt.vm b/hadoop-common-project/hadoop-common/src/site/apt/NativeLibraries.apt.vm
index 49818af..866b428 100644
--- a/hadoop-common-project/hadoop-common/src/site/apt/NativeLibraries.apt.vm
+++ b/hadoop-common-project/hadoop-common/src/site/apt/NativeLibraries.apt.vm
@@ -164,6 +164,24 @@ Native Libraries Guide
      * If something goes wrong, then:
        <<<INFO util.NativeCodeLoader - Unable to load native-hadoop library for your platform... using builtin-java classes where applicable>>>
 
+* Check
+
+   NativeLibraryChecker is a tool to check whether native libraries are loaded correctly.
+   You can launch NativeLibraryChecker as follows:
+
+----
+   $ hadoop checknative -a
+   14/12/06 01:30:45 WARN bzip2.Bzip2Factory: Failed to load/initialize native-bzip2 library system-native, will use pure-Java version
+   14/12/06 01:30:45 INFO zlib.ZlibFactory: Successfully loaded & initialized native-zlib library
+   Native library checking:
+   hadoop: true /home/ozawa/hadoop/lib/native/libhadoop.so.1.0.0
+   zlib:   true /lib/x86_64-linux-gnu/libz.so.1
+   snappy: true /usr/lib/libsnappy.so.1
+   lz4:    true revision:99
+   bzip2:  false
+----
+
+
 * Native Shared Libraries
 
    You can load any native shared library using DistributedCache for


[20/29] hadoop git commit: YARN-2461. Fix PROCFS_USE_SMAPS_BASED_RSS_ENABLED property in YarnConfiguration. (rchiang via rkanter)

Posted by vi...@apache.org.
YARN-2461. Fix PROCFS_USE_SMAPS_BASED_RSS_ENABLED property in YarnConfiguration. (rchiang via rkanter)


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

Branch: refs/heads/HDFS-EC
Commit: 3c72f54ef581b4f3e2eb84e1e24e459c38d3f769
Parents: 9cdaec6
Author: Robert Kanter <rk...@apache.org>
Authored: Fri Dec 5 12:07:01 2014 -0800
Committer: Robert Kanter <rk...@apache.org>
Committed: Fri Dec 5 12:07:41 2014 -0800

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                                   | 3 +++
 .../main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java  | 2 +-
 2 files changed, 4 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/3c72f54e/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index 252b7d5..9804d61 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -189,6 +189,9 @@ Release 2.7.0 - UNRELEASED
     YARN-2874. Dead lock in "DelegationTokenRenewer" which blocks RM to execute
     any further apps. (Naganarasimha G R via kasha)
 
+    YARN-2461. Fix PROCFS_USE_SMAPS_BASED_RSS_ENABLED property in
+    YarnConfiguration. (rchiang via rkanter)
+
 Release 2.6.0 - 2014-11-18
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3c72f54e/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 f0f88d8..10ba832 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
@@ -819,7 +819,7 @@ public class YarnConfiguration extends Configuration {
   public static final String NM_CONTAINER_MON_PROCESS_TREE =
     NM_PREFIX + "container-monitor.process-tree.class";
   public static final String PROCFS_USE_SMAPS_BASED_RSS_ENABLED = NM_PREFIX +
-      ".container-monitor.procfs-tree.smaps-based-rss.enabled";
+      "container-monitor.procfs-tree.smaps-based-rss.enabled";
   public static final boolean DEFAULT_PROCFS_USE_SMAPS_BASED_RSS_ENABLED =
       false;
   


[29/29] hadoop git commit: Merge remote-tracking branch 'origin/trunk' into HDFS-EC

Posted by vi...@apache.org.
Merge remote-tracking branch 'origin/trunk' into HDFS-EC


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

Branch: refs/heads/HDFS-EC
Commit: 8331853450bc52fb3d8eed61fde52639a5fdb619
Parents: bdc0101 120e1de
Author: Vinayakumar B <vi...@intel.com>
Authored: Mon Dec 8 14:42:52 2014 +0530
Committer: Vinayakumar B <vi...@intel.com>
Committed: Mon Dec 8 14:42:52 2014 +0530

----------------------------------------------------------------------
 .../client/KerberosAuthenticator.java           |   6 +-
 hadoop-common-project/hadoop-common/CHANGES.txt |  21 +
 .../hadoop-common/src/CMakeLists.txt            |   2 +-
 .../apache/hadoop/crypto/AesCtrCryptoCodec.java |  27 +-
 .../fs/permission/AccessControlException.java   |  66 ---
 .../hadoop/security/AccessControlException.java |   4 +-
 .../src/site/apt/NativeLibraries.apt.vm         |  18 +
 .../apache/hadoop/crypto/TestCryptoCodec.java   |  64 +++
 .../hadoop/crypto/key/kms/server/KMSACLs.java   |  26 +-
 .../kms/server/KeyAuthorizationKeyProvider.java |   4 +
 .../hadoop/crypto/key/kms/server/TestKMS.java   |  13 +-
 hadoop-hdfs-project/hadoop-hdfs-nfs/pom.xml     |   5 +
 .../hadoop/hdfs/nfs/conf/NfsConfigKeys.java     |  10 +
 .../hadoop/hdfs/nfs/nfs3/Nfs3HttpServer.java    | 111 ++++
 .../hadoop/hdfs/nfs/nfs3/RpcProgramNfs3.java    |  24 +-
 .../hdfs/nfs/nfs3/TestNfs3HttpServer.java       |  89 ++++
 .../src/test/resources/core-site.xml            |  14 +
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |  31 +-
 hadoop-hdfs-project/hadoop-hdfs/pom.xml         |   3 +
 .../datanode/ReplicaNotFoundException.java      |   2 +-
 .../server/namenode/AclEntryStatusFormat.java   | 136 +++++
 .../hadoop/hdfs/server/namenode/AclFeature.java |  24 +-
 .../hadoop/hdfs/server/namenode/AclStorage.java |  61 +--
 .../server/namenode/EncryptionZoneManager.java  |   4 +-
 .../hadoop/hdfs/server/namenode/FSDirAclOp.java | 244 +++++++++
 .../hdfs/server/namenode/FSDirConcatOp.java     |   9 +-
 .../hdfs/server/namenode/FSDirMkdirOp.java      |  17 +-
 .../hdfs/server/namenode/FSDirRenameOp.java     |  74 ++-
 .../hdfs/server/namenode/FSDirSnapshotOp.java   |  68 ++-
 .../server/namenode/FSDirStatAndListingOp.java  |  35 +-
 .../hdfs/server/namenode/FSDirectory.java       | 329 +++---------
 .../hdfs/server/namenode/FSEditLogLoader.java   |  18 +-
 .../server/namenode/FSImageFormatPBINode.java   |  22 +-
 .../hdfs/server/namenode/FSNDNCacheOp.java      | 124 +++++
 .../hdfs/server/namenode/FSNamesystem.java      | 531 +++++++------------
 .../server/namenode/FSPermissionChecker.java    |  61 +--
 .../hdfs/server/namenode/INodesInPath.java      |  21 +-
 .../hadoop/hdfs/server/namenode/NNConf.java     | 104 ----
 .../snapshot/FSImageFormatPBSnapshot.java       |  13 +-
 .../namenode/snapshot/SnapshotManager.java      |  50 +-
 .../hdfs/server/namenode/FSAclBaseTest.java     |   4 +-
 .../hdfs/server/namenode/TestAuditLogger.java   |  79 +--
 .../namenode/TestFSPermissionChecker.java       |  10 +-
 .../server/namenode/TestSnapshotPathINodes.java |  20 +-
 .../namenode/snapshot/TestSnapshotManager.java  |  14 +-
 hadoop-mapreduce-project/CHANGES.txt            |   3 +
 .../apache/hadoop/mapred/MapReduceChildJVM.java |  34 +-
 .../v2/app/job/impl/TestMapReduceChildJVM.java  |  71 ++-
 .../apache/hadoop/mapreduce/v2/util/MRApps.java |  80 ++-
 .../apache/hadoop/mapred/FileOutputFormat.java  |   4 +-
 .../java/org/apache/hadoop/mapred/TaskLog.java  |   4 +
 .../apache/hadoop/mapreduce/MRJobConfig.java    |  14 +
 .../src/main/resources/mapred-default.xml       |  28 +
 .../org/apache/hadoop/mapred/YARNRunner.java    |   9 +-
 hadoop-yarn-project/CHANGES.txt                 |  28 +
 hadoop-yarn-project/hadoop-yarn/bin/yarn        |   5 +
 .../hadoop-yarn/hadoop-yarn-api/pom.xml         |   1 +
 .../hadoop/yarn/conf/YarnConfiguration.java     |  16 +-
 .../yarn/server/api/SCMAdminProtocol.java       |  53 ++
 .../yarn/server/api/SCMAdminProtocolPB.java     |  31 ++
 .../RunSharedCacheCleanerTaskRequest.java       |  37 ++
 .../RunSharedCacheCleanerTaskResponse.java      |  58 ++
 .../main/proto/server/SCM_Admin_protocol.proto  |  29 +
 .../src/main/proto/yarn_service_protos.proto    |  11 +
 .../org/apache/hadoop/yarn/client/SCMAdmin.java | 183 +++++++
 .../hadoop/yarn/client/cli/ApplicationCLI.java  |   8 +-
 .../hadoop/yarn/client/cli/TestYarnCLI.java     |  41 +-
 .../hadoop/yarn/ContainerLogAppender.java       |  11 +-
 .../yarn/ContainerRollingLogAppender.java       |  11 +-
 .../pb/client/SCMAdminProtocolPBClientImpl.java |  73 +++
 .../service/SCMAdminProtocolPBServiceImpl.java  |  57 ++
 .../RunSharedCacheCleanerTaskRequestPBImpl.java |  53 ++
 ...RunSharedCacheCleanerTaskResponsePBImpl.java |  66 +++
 .../src/main/resources/yarn-default.xml         |  12 +
 .../hadoop/yarn/TestContainerLogAppender.java   |   1 +
 .../nodemanager/metrics/NodeManagerMetrics.java |  19 +-
 .../impl/container-executor.c                   |   2 +-
 .../main/resources/container-log4j.properties   |  29 +-
 .../metrics/TestNodeManagerMetrics.java         |  17 +-
 .../yarn/server/resourcemanager/RMContext.java  |   3 +
 .../server/resourcemanager/RMContextImpl.java   |  11 +
 .../server/resourcemanager/ResourceManager.java |   2 +
 .../ProportionalCapacityPreemptionPolicy.java   | 170 +++++-
 .../rmcontainer/RMContainerImpl.java            |   9 +-
 .../CapacitySchedulerConfiguration.java         |  10 +-
 .../security/DelegationTokenRenewer.java        |  12 +-
 .../resourcemanager/TestClientRMService.java    |   1 +
 .../server/resourcemanager/TestRMRestart.java   |  91 ++++
 ...estProportionalCapacityPreemptionPolicy.java | 283 +++++++++-
 .../rmcontainer/TestRMContainerImpl.java        |   6 +-
 .../scheduler/capacity/TestQueueParsing.java    | 110 ++++
 .../SCMAdminProtocolService.java                | 146 +++++
 .../sharedcachemanager/SharedCacheManager.java  |   8 +
 .../TestSCMAdminProtocolService.java            | 135 +++++
 94 files changed, 3501 insertions(+), 1207 deletions(-)
----------------------------------------------------------------------



[07/29] hadoop git commit: YARN-2891. Failed Container Executor does not provide a clear error message. Contributed by Dustin Cote. (harsh)

Posted by vi...@apache.org.
YARN-2891. Failed Container Executor does not provide a clear error message. Contributed by Dustin Cote. (harsh)


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

Branch: refs/heads/HDFS-EC
Commit: a31e0164912236630c485e5aeb908b43e3a67c61
Parents: 799353e
Author: Harsh J <ha...@cloudera.com>
Authored: Thu Dec 4 03:16:08 2014 +0530
Committer: Harsh J <ha...@cloudera.com>
Committed: Thu Dec 4 03:17:15 2014 +0530

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                                   | 3 +++
 .../src/main/native/container-executor/impl/container-executor.c  | 2 +-
 2 files changed, 4 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/a31e0164/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index d44f46d..91151ad 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -59,6 +59,9 @@ Release 2.7.0 - UNRELEASED
 
   IMPROVEMENTS
 
+    YARN-2891. Failed Container Executor does not provide a clear error
+    message. (Dustin Cote via harsh)
+
     YARN-1979. TestDirectoryCollection fails when the umask is unusual.
     (Vinod Kumar Vavilapalli and Tsuyoshi OZAWA via junping_du)
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a31e0164/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/container-executor.c
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/container-executor.c b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/container-executor.c
index 9af9161..4fc78b6 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/container-executor.c
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/container-executor.c
@@ -526,7 +526,7 @@ int check_dir(char* npath, mode_t st_mode, mode_t desired, int finalComponent) {
     int filePermInt = st_mode & (S_IRWXU | S_IRWXG | S_IRWXO);
     int desiredInt = desired & (S_IRWXU | S_IRWXG | S_IRWXO);
     if (filePermInt != desiredInt) {
-      fprintf(LOGFILE, "Path %s does not have desired permission.\n", npath);
+      fprintf(LOGFILE, "Path %s has permission %o but needs permission %o.\n", npath, filePermInt, desiredInt);
       return -1;
     }
   }


[03/29] hadoop git commit: MAPREDUCE-5932. Provide an option to use a dedicated reduce-side shuffle log. Contributed by Gera Shegalov

Posted by vi...@apache.org.
MAPREDUCE-5932. Provide an option to use a dedicated reduce-side shuffle log. 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/03ab24aa
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/03ab24aa
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/03ab24aa

Branch: refs/heads/HDFS-EC
Commit: 03ab24aa01ffea1cacf1fa9cbbf73c3f2904d981
Parents: 22afae8
Author: Jason Lowe <jl...@apache.org>
Authored: Wed Dec 3 17:02:14 2014 +0000
Committer: Jason Lowe <jl...@apache.org>
Committed: Wed Dec 3 17:02:14 2014 +0000

----------------------------------------------------------------------
 hadoop-mapreduce-project/CHANGES.txt            |  3 +
 .../apache/hadoop/mapred/MapReduceChildJVM.java | 34 +--------
 .../v2/app/job/impl/TestMapReduceChildJVM.java  | 71 ++++++++++++++++-
 .../apache/hadoop/mapreduce/v2/util/MRApps.java | 80 +++++++++++++++++---
 .../apache/hadoop/mapred/FileOutputFormat.java  |  4 +-
 .../java/org/apache/hadoop/mapred/TaskLog.java  |  4 +
 .../apache/hadoop/mapreduce/MRJobConfig.java    | 14 ++++
 .../src/main/resources/mapred-default.xml       | 28 +++++++
 .../org/apache/hadoop/mapred/YARNRunner.java    |  9 +--
 .../hadoop/yarn/ContainerLogAppender.java       | 11 ++-
 .../yarn/ContainerRollingLogAppender.java       | 11 ++-
 .../hadoop/yarn/TestContainerLogAppender.java   |  1 +
 .../main/resources/container-log4j.properties   | 29 ++++++-
 13 files changed, 243 insertions(+), 56 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/03ab24aa/hadoop-mapreduce-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/CHANGES.txt b/hadoop-mapreduce-project/CHANGES.txt
index 5417c3e..3f34acd 100644
--- a/hadoop-mapreduce-project/CHANGES.txt
+++ b/hadoop-mapreduce-project/CHANGES.txt
@@ -235,6 +235,9 @@ Release 2.7.0 - UNRELEASED
 
   IMPROVEMENTS
 
+    MAPREDUCE-5932. Provide an option to use a dedicated reduce-side shuffle
+    log (Gera Shegalov via jlowe)
+
   OPTIMIZATIONS
 
     MAPREDUCE-6169. MergeQueue should release reference to the current item 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/03ab24aa/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapred/MapReduceChildJVM.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapred/MapReduceChildJVM.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapred/MapReduceChildJVM.java
index c790c57..817b3a5 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapred/MapReduceChildJVM.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapred/MapReduceChildJVM.java
@@ -20,16 +20,14 @@ package org.apache.hadoop.mapred;
 
 import java.net.InetSocketAddress;
 import java.util.HashMap;
-import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 import java.util.Vector;
 
-import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.mapred.TaskLog.LogName;
-import org.apache.hadoop.mapreduce.ID;
 import org.apache.hadoop.mapreduce.MRJobConfig;
+import org.apache.hadoop.mapreduce.TypeConverter;
 import org.apache.hadoop.mapreduce.v2.util.MRApps;
 import org.apache.hadoop.yarn.api.ApplicationConstants;
 import org.apache.hadoop.yarn.api.ApplicationConstants.Environment;
@@ -52,20 +50,6 @@ public class MapReduceChildJVM {
         jobConf.get(JobConf.MAPRED_TASK_ENV));
   }
 
-  private static String getChildLogLevel(JobConf conf, boolean isMap) {
-    if (isMap) {
-      return conf.get(
-          MRJobConfig.MAP_LOG_LEVEL, 
-          JobConf.DEFAULT_LOG_LEVEL.toString()
-          );
-    } else {
-      return conf.get(
-          MRJobConfig.REDUCE_LOG_LEVEL, 
-          JobConf.DEFAULT_LOG_LEVEL.toString()
-          );
-    }
-  }
-  
   public static void setVMEnv(Map<String, String> environment,
       Task task) {
 
@@ -79,7 +63,7 @@ public class MapReduceChildJVM {
     // streaming) it will have the correct loglevel.
     environment.put(
         "HADOOP_ROOT_LOGGER", 
-        getChildLogLevel(conf, task.isMapTask()) + ",console");
+        MRApps.getChildLogLevel(conf, task.isMapTask()) + ",console");
 
     // TODO: The following is useful for instance in streaming tasks. Should be
     // set in ApplicationMaster's env by the RM.
@@ -147,15 +131,6 @@ public class MapReduceChildJVM {
     return adminClasspath + " " + userClasspath;
   }
 
-  private static void setupLog4jProperties(Task task,
-      Vector<String> vargs,
-      long logSize, Configuration conf) {
-    String logLevel = getChildLogLevel(task.conf, task.isMapTask());
-    int numBackups = task.conf.getInt(MRJobConfig.TASK_LOG_BACKUPS,
-        MRJobConfig.DEFAULT_TASK_LOG_BACKUPS);
-    MRApps.addLog4jSystemProperties(logLevel, logSize, numBackups, vargs, conf);
-  }
-
   public static List<String> getVMCommand(
       InetSocketAddress taskAttemptListenerAddr, Task task, 
       JVMId jvmID) {
@@ -206,10 +181,7 @@ public class MapReduceChildJVM {
     Path childTmpDir = new Path(MRApps.crossPlatformifyMREnv(conf, Environment.PWD),
         YarnConfiguration.DEFAULT_CONTAINER_TEMP_DIR);
     vargs.add("-Djava.io.tmpdir=" + childTmpDir);
-
-    // Setup the log4j prop
-    long logSize = TaskLog.getTaskLogLength(conf);
-    setupLog4jProperties(task, vargs, logSize, conf);
+    MRApps.addLog4jSystemProperties(task, vargs, conf);
 
     if (conf.getProfileEnabled()) {
       if (conf.getProfileTaskRange(task.isMapTask()

http://git-wip-us.apache.org/repos/asf/hadoop/blob/03ab24aa/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TestMapReduceChildJVM.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TestMapReduceChildJVM.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TestMapReduceChildJVM.java
index 8e146b9..b1e9cf0 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TestMapReduceChildJVM.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TestMapReduceChildJVM.java
@@ -61,7 +61,7 @@ public class TestMapReduceChildJVM {
       " -Dlog4j.configuration=container-log4j.properties" +
       " -Dyarn.app.container.log.dir=<LOG_DIR>" +
       " -Dyarn.app.container.log.filesize=0" +
-      " -Dhadoop.root.logger=INFO,CLA" +
+      " -Dhadoop.root.logger=INFO,CLA -Dhadoop.root.logfile=syslog" +
       " org.apache.hadoop.mapred.YarnChild 127.0.0.1" +
       " 54321" +
       " attempt_0_0000_m_000000_0" +
@@ -77,6 +77,73 @@ public class TestMapReduceChildJVM {
       app.cmdEnvironment.containsKey("HADOOP_CLIENT_OPTS"));
     Assert.assertEquals("", app.cmdEnvironment.get("HADOOP_CLIENT_OPTS"));
   }
+
+  @Test (timeout = 30000)
+  public void testReduceCommandLineWithSeparateShuffle() throws Exception {
+    final Configuration conf = new Configuration();
+    conf.setBoolean(MRJobConfig.REDUCE_SEPARATE_SHUFFLE_LOG, true);
+    testReduceCommandLine(conf);
+  }
+
+  @Test (timeout = 30000)
+  public void testReduceCommandLineWithSeparateCRLAShuffle() throws Exception {
+    final Configuration conf = new Configuration();
+    conf.setBoolean(MRJobConfig.REDUCE_SEPARATE_SHUFFLE_LOG, true);
+    conf.setLong(MRJobConfig.SHUFFLE_LOG_KB, 1L);
+    conf.setInt(MRJobConfig.SHUFFLE_LOG_BACKUPS, 3);
+    testReduceCommandLine(conf);
+  }
+
+  @Test (timeout = 30000)
+  public void testReduceCommandLine() throws Exception {
+    final Configuration conf = new Configuration();
+    testReduceCommandLine(conf);
+  }
+
+  private void testReduceCommandLine(Configuration conf)
+      throws Exception {
+
+    MyMRApp app = new MyMRApp(0, 1, true, this.getClass().getName(), true);
+    conf.setBoolean(MRConfig.MAPREDUCE_APP_SUBMISSION_CROSS_PLATFORM, true);
+    Job job = app.submit(conf);
+    app.waitForState(job, JobState.SUCCEEDED);
+    app.verifyCompleted();
+
+    final long shuffleLogSize =
+        conf.getLong(MRJobConfig.SHUFFLE_LOG_KB, 0L) * 1024L;
+    final int shuffleBackups = conf.getInt(MRJobConfig.SHUFFLE_LOG_BACKUPS, 0);
+    final String appenderName = shuffleLogSize > 0L && shuffleBackups > 0
+        ? "shuffleCRLA"
+        : "shuffleCLA";
+
+    Assert.assertEquals(
+        "[" + MRApps.crossPlatformify("JAVA_HOME") + "/bin/java" +
+            " -Djava.net.preferIPv4Stack=true" +
+            " -Dhadoop.metrics.log.level=WARN" +
+            "  -Xmx200m -Djava.io.tmpdir=" + MRApps.crossPlatformify("PWD") + "/tmp" +
+            " -Dlog4j.configuration=container-log4j.properties" +
+            " -Dyarn.app.container.log.dir=<LOG_DIR>" +
+            " -Dyarn.app.container.log.filesize=0" +
+            " -Dhadoop.root.logger=INFO,CLA -Dhadoop.root.logfile=syslog" +
+            " -Dyarn.app.mapreduce.shuffle.logger=INFO," + appenderName +
+            " -Dyarn.app.mapreduce.shuffle.logfile=syslog.shuffle" +
+            " -Dyarn.app.mapreduce.shuffle.log.filesize=" + shuffleLogSize +
+            " -Dyarn.app.mapreduce.shuffle.log.backups=" + shuffleBackups +
+            " org.apache.hadoop.mapred.YarnChild 127.0.0.1" +
+            " 54321" +
+            " attempt_0_0000_r_000000_0" +
+            " 0" +
+            " 1><LOG_DIR>/stdout" +
+            " 2><LOG_DIR>/stderr ]", app.myCommandLine);
+
+    Assert.assertTrue("HADOOP_ROOT_LOGGER not set for job",
+        app.cmdEnvironment.containsKey("HADOOP_ROOT_LOGGER"));
+    Assert.assertEquals("INFO,console",
+        app.cmdEnvironment.get("HADOOP_ROOT_LOGGER"));
+    Assert.assertTrue("HADOOP_CLIENT_OPTS not set for job",
+        app.cmdEnvironment.containsKey("HADOOP_CLIENT_OPTS"));
+    Assert.assertEquals("", app.cmdEnvironment.get("HADOOP_CLIENT_OPTS"));
+  }
   
   @Test (timeout = 30000)
   public void testCommandLineWithLog4JConifg() throws Exception {
@@ -99,7 +166,7 @@ public class TestMapReduceChildJVM {
       " -Dlog4j.configuration=" + testLogPropertieFile +
       " -Dyarn.app.container.log.dir=<LOG_DIR>" +
       " -Dyarn.app.container.log.filesize=0" +
-      " -Dhadoop.root.logger=INFO,CLA" +
+      " -Dhadoop.root.logger=INFO,CLA -Dhadoop.root.logfile=syslog" +
       " org.apache.hadoop.mapred.YarnChild 127.0.0.1" +
       " 54321" +
       " attempt_0_0000_m_000000_0" +

http://git-wip-us.apache.org/repos/asf/hadoop/blob/03ab24aa/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapreduce/v2/util/MRApps.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapreduce/v2/util/MRApps.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapreduce/v2/util/MRApps.java
index 4484e6a..08b44f8 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapreduce/v2/util/MRApps.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapreduce/v2/util/MRApps.java
@@ -43,6 +43,9 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.mapred.InvalidJobConfException;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.Task;
+import org.apache.hadoop.mapred.TaskLog;
 import org.apache.hadoop.mapreduce.JobID;
 import org.apache.hadoop.mapreduce.MRConfig;
 import org.apache.hadoop.mapreduce.MRJobConfig;
@@ -59,6 +62,7 @@ import org.apache.hadoop.mapreduce.v2.api.records.TaskType;
 import org.apache.hadoop.util.ApplicationClassLoader;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.yarn.ContainerLogAppender;
+import org.apache.hadoop.yarn.ContainerRollingLogAppender;
 import org.apache.hadoop.yarn.api.ApplicationConstants;
 import org.apache.hadoop.yarn.api.ApplicationConstants.Environment;
 import org.apache.hadoop.yarn.api.records.LocalResource;
@@ -68,7 +72,6 @@ import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
 import org.apache.hadoop.yarn.util.Apps;
 import org.apache.hadoop.yarn.util.ConverterUtils;
-import org.apache.log4j.RollingFileAppender;
 
 /**
  * Helper class for MR applications
@@ -592,18 +595,32 @@ public class MRApps extends Apps {
     }
     return result;
   }
+
+  public static String getChildLogLevel(Configuration conf, boolean isMap) {
+    if (isMap) {
+      return conf.get(
+          MRJobConfig.MAP_LOG_LEVEL,
+          JobConf.DEFAULT_LOG_LEVEL.toString()
+      );
+    } else {
+      return conf.get(
+          MRJobConfig.REDUCE_LOG_LEVEL,
+          JobConf.DEFAULT_LOG_LEVEL.toString()
+      );
+    }
+  }
   
   /**
-   * Add the JVM system properties necessary to configure {@link ContainerLogAppender}.
-   * @param logLevel the desired log level (eg INFO/WARN/DEBUG)
-   * @param logSize See {@link ContainerLogAppender#setTotalLogFileSize(long)}
-   * @param numBackups See {@link RollingFileAppender#setMaxBackupIndex(int)}
+   * Add the JVM system properties necessary to configure
+   *  {@link ContainerLogAppender} or
+   *  {@link ContainerRollingLogAppender}.
+   *
+   * @param task for map/reduce, or null for app master
    * @param vargs the argument list to append to
    * @param conf configuration of MR job
    */
-  public static void addLog4jSystemProperties(
-      String logLevel, long logSize, int numBackups, List<String> vargs, 
-      Configuration conf) {
+  public static void addLog4jSystemProperties(Task task,
+      List<String> vargs, Configuration conf) {
     String log4jPropertyFile =
         conf.get(MRJobConfig.MAPREDUCE_JOB_LOG4J_PROPERTIES_FILE, "");
     if (log4jPropertyFile.isEmpty()) {
@@ -618,11 +635,30 @@ public class MRApps extends Apps {
       Path log4jPath = new Path(log4jURI);
       vargs.add("-Dlog4j.configuration="+log4jPath.getName());
     }
-    
+
+    long logSize;
+    String logLevel;
+    int numBackups;
+
+    if (task == null) {
+      logSize = conf.getLong(MRJobConfig.MR_AM_LOG_KB,
+          MRJobConfig.DEFAULT_MR_AM_LOG_KB) << 10;
+      logLevel = conf.get(
+          MRJobConfig.MR_AM_LOG_LEVEL, MRJobConfig.DEFAULT_MR_AM_LOG_LEVEL);
+      numBackups = conf.getInt(MRJobConfig.MR_AM_LOG_BACKUPS,
+          MRJobConfig.DEFAULT_MR_AM_LOG_BACKUPS);
+    } else {
+      logSize = TaskLog.getTaskLogLimitBytes(conf);
+      logLevel = getChildLogLevel(conf, task.isMapTask());
+      numBackups = conf.getInt(MRJobConfig.TASK_LOG_BACKUPS,
+          MRJobConfig.DEFAULT_TASK_LOG_BACKUPS);
+    }
+
     vargs.add("-D" + YarnConfiguration.YARN_APP_CONTAINER_LOG_DIR + "=" +
         ApplicationConstants.LOG_DIR_EXPANSION_VAR);
     vargs.add(
         "-D" + YarnConfiguration.YARN_APP_CONTAINER_LOG_SIZE + "=" + logSize);
+
     if (logSize > 0L && numBackups > 0) {
       // log should be rolled
       vargs.add("-D" + YarnConfiguration.YARN_APP_CONTAINER_LOG_BACKUPS + "="
@@ -631,6 +667,30 @@ public class MRApps extends Apps {
     } else {
       vargs.add("-Dhadoop.root.logger=" + logLevel + ",CLA");
     }
+    vargs.add("-Dhadoop.root.logfile=" + TaskLog.LogName.SYSLOG);
+
+    if (   task != null
+        && !task.isMapTask()
+        && conf.getBoolean(MRJobConfig.REDUCE_SEPARATE_SHUFFLE_LOG,
+               MRJobConfig.DEFAULT_REDUCE_SEPARATE_SHUFFLE_LOG)) {
+      final int numShuffleBackups = conf.getInt(MRJobConfig.SHUFFLE_LOG_BACKUPS,
+          MRJobConfig.DEFAULT_SHUFFLE_LOG_BACKUPS);
+      final long shuffleLogSize = conf.getLong(MRJobConfig.SHUFFLE_LOG_KB,
+          MRJobConfig.DEFAULT_SHUFFLE_LOG_KB) << 10;
+      final String shuffleLogger = logLevel
+          + (shuffleLogSize > 0L && numShuffleBackups > 0
+                 ? ",shuffleCRLA"
+                 : ",shuffleCLA");
+
+      vargs.add("-D" + MRJobConfig.MR_PREFIX
+          + "shuffle.logger=" + shuffleLogger);
+      vargs.add("-D" + MRJobConfig.MR_PREFIX
+          + "shuffle.logfile=" + TaskLog.LogName.SYSLOG + ".shuffle");
+      vargs.add("-D" + MRJobConfig.MR_PREFIX
+          + "shuffle.log.filesize=" + shuffleLogSize);
+      vargs.add("-D" + MRJobConfig.MR_PREFIX
+          + "shuffle.log.backups=" + numShuffleBackups);
+    }
   }
 
   /**
@@ -687,7 +747,7 @@ public class MRApps extends Apps {
   public static String crossPlatformifyMREnv(Configuration conf, Environment env) {
     boolean crossPlatform =
         conf.getBoolean(MRConfig.MAPREDUCE_APP_SUBMISSION_CROSS_PLATFORM,
-          MRConfig.DEFAULT_MAPREDUCE_APP_SUBMISSION_CROSS_PLATFORM);
+            MRConfig.DEFAULT_MAPREDUCE_APP_SUBMISSION_CROSS_PLATFORM);
     return crossPlatform ? env.$$() : env.$();
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/03ab24aa/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/FileOutputFormat.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/FileOutputFormat.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/FileOutputFormat.java
index 0efcf9d..721c8a8 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/FileOutputFormat.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/FileOutputFormat.java
@@ -27,7 +27,6 @@ import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.compress.CompressionCodec;
-import org.apache.hadoop.mapred.FileAlreadyExistsException;
 import org.apache.hadoop.mapreduce.security.TokenCache;
 import org.apache.hadoop.util.Progressable;
 
@@ -287,7 +286,8 @@ public abstract class FileOutputFormat<K, V> implements OutputFormat<K, V> {
         "This method can only be called from within a Job");
     }
 
-    String taskType = (conf.getBoolean(JobContext.TASK_ISMAP, true)) ? "m" : "r";
+    String taskType = conf.getBoolean(JobContext.TASK_ISMAP,
+        JobContext.DEFAULT_TASK_ISMAP) ? "m" : "r";
 
     NumberFormat numberFormat = NumberFormat.getInstance();
     numberFormat.setMinimumIntegerDigits(5);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/03ab24aa/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/TaskLog.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/TaskLog.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/TaskLog.java
index a86e76a..e07b5be 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/TaskLog.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/TaskLog.java
@@ -473,6 +473,10 @@ public class TaskLog {
    * @return the number of bytes to cap the log files at
    */
   public static long getTaskLogLength(JobConf conf) {
+   return getTaskLogLimitBytes(conf);
+  }
+
+  public static long getTaskLogLimitBytes(Configuration conf) {
     return conf.getLong(JobContext.TASK_USERLOG_LIMIT, 0) * 1024;
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/03ab24aa/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/MRJobConfig.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/MRJobConfig.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/MRJobConfig.java
index 691074a..230361c 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/MRJobConfig.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/MRJobConfig.java
@@ -177,6 +177,7 @@ public interface MRJobConfig {
   public static final String TASK_ATTEMPT_ID = "mapreduce.task.attempt.id";
 
   public static final String TASK_ISMAP = "mapreduce.task.ismap";
+  public static final boolean DEFAULT_TASK_ISMAP = true;
 
   public static final String TASK_PARTITION = "mapreduce.task.partition";
 
@@ -773,6 +774,18 @@ public interface MRJobConfig {
       MR_PREFIX + "task.container.log.backups";
   public static final int DEFAULT_TASK_LOG_BACKUPS = 0; // don't roll
 
+  public static final String REDUCE_SEPARATE_SHUFFLE_LOG =
+      MR_PREFIX + "shuffle.log.separate";
+  public static final boolean DEFAULT_REDUCE_SEPARATE_SHUFFLE_LOG = true;
+
+  public static final String SHUFFLE_LOG_BACKUPS =
+      MR_PREFIX + "shuffle.log.backups";
+  public static final int DEFAULT_SHUFFLE_LOG_BACKUPS = 0; // don't roll
+
+  public static final String SHUFFLE_LOG_KB =
+      MR_PREFIX + "shuffle.log.limit.kb";
+  public static final long DEFAULT_SHUFFLE_LOG_KB = 0L;
+
   public static final String WORKFLOW_NAME = "mapreduce.workflow.name";
   
   public static final String WORKFLOW_NODE_NAME =
@@ -812,4 +825,5 @@ public interface MRJobConfig {
       "mapreduce.job.encrypted-intermediate-data.buffer.kb";
   public static final int DEFAULT_MR_ENCRYPTED_INTERMEDIATE_DATA_BUFFER_KB =
           128;
+
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/03ab24aa/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/resources/mapred-default.xml
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/resources/mapred-default.xml b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/resources/mapred-default.xml
index 6be62ec..43ddb13 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/resources/mapred-default.xml
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/resources/mapred-default.xml
@@ -714,6 +714,34 @@
 </property>
 
 <property>
+  <name>yarn.app.mapreduce.shuffle.log.separate</name>
+  <value>true</value>
+  <description>If enabled ('true') logging generated by the client-side shuffle
+    classes in a reducer will be written in a dedicated log file
+    'syslog.shuffle' instead of 'syslog'.
+  </description>
+</property>
+
+<property>
+  <name>yarn.app.mapreduce.shuffle.log.limit.kb</name>
+  <value>0</value>
+  <description>Maximum size of the syslog.shuffle file in kilobytes
+    (0 for no limit).
+  </description>
+</property>
+
+<property>
+  <name>yarn.app.mapreduce.shuffle.log.backups</name>
+  <value>0</value>
+  <description>If yarn.app.mapreduce.shuffle.log.limit.kb and
+    yarn.app.mapreduce.shuffle.log.backups are greater than zero
+    then a ContainerRollngLogAppender is used instead of ContainerLogAppender
+    for syslog.shuffle. See
+    org.apache.log4j.RollingFileAppender.maxBackupIndex
+  </description>
+</property>
+
+<property>
   <name>mapreduce.job.maxtaskfailures.per.tracker</name>
   <value>3</value>
   <description>The number of task-failures on a node manager of a given job 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/03ab24aa/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/main/java/org/apache/hadoop/mapred/YARNRunner.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/main/java/org/apache/hadoop/mapred/YARNRunner.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/main/java/org/apache/hadoop/mapred/YARNRunner.java
index 7b2cf53..41dc72f 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/main/java/org/apache/hadoop/mapred/YARNRunner.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/main/java/org/apache/hadoop/mapred/YARNRunner.java
@@ -392,14 +392,7 @@ public class YARNRunner implements ClientProtocol {
     vargs.add(MRApps.crossPlatformifyMREnv(jobConf, Environment.JAVA_HOME)
         + "/bin/java");
 
-    // TODO: why do we use 'conf' some places and 'jobConf' others?
-    long logSize = jobConf.getLong(MRJobConfig.MR_AM_LOG_KB,
-        MRJobConfig.DEFAULT_MR_AM_LOG_KB) << 10;
-    String logLevel = jobConf.get(
-        MRJobConfig.MR_AM_LOG_LEVEL, MRJobConfig.DEFAULT_MR_AM_LOG_LEVEL);
-    int numBackups = jobConf.getInt(MRJobConfig.MR_AM_LOG_BACKUPS,
-        MRJobConfig.DEFAULT_MR_AM_LOG_BACKUPS);
-    MRApps.addLog4jSystemProperties(logLevel, logSize, numBackups, vargs, conf);
+    MRApps.addLog4jSystemProperties(null, vargs, conf);
 
     // Check for Java Lib Path usage in MAP and REDUCE configs
     warnForJavaLibPath(conf.get(MRJobConfig.MAP_JAVA_OPTS,""), "map", 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/03ab24aa/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/ContainerLogAppender.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/ContainerLogAppender.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/ContainerLogAppender.java
index 0aba8b5..c49a1ab 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/ContainerLogAppender.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/ContainerLogAppender.java
@@ -38,6 +38,7 @@ public class ContainerLogAppender extends FileAppender
   implements Flushable
 {
   private String containerLogDir;
+  private String containerLogFile;
   //so that log4j can configure it from the configuration(log4j.properties). 
   private int maxEvents;
   private Queue<LoggingEvent> tail = null;
@@ -49,7 +50,7 @@ public class ContainerLogAppender extends FileAppender
       if (maxEvents > 0) {
         tail = new LinkedList<LoggingEvent>();
       }
-      setFile(new File(this.containerLogDir, "syslog").toString());
+      setFile(new File(this.containerLogDir, containerLogFile).toString());
       setAppend(true);
       super.activateOptions();
     }
@@ -102,6 +103,14 @@ public class ContainerLogAppender extends FileAppender
     this.containerLogDir = containerLogDir;
   }
 
+  public String getContainerLogFile() {
+    return containerLogFile;
+  }
+
+  public void setContainerLogFile(String containerLogFile) {
+    this.containerLogFile = containerLogFile;
+  }
+
   private static final int EVENT_SIZE = 100;
   
   public long getTotalLogFileSize() {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/03ab24aa/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/ContainerRollingLogAppender.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/ContainerRollingLogAppender.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/ContainerRollingLogAppender.java
index bdf1b09..7dd712e 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/ContainerRollingLogAppender.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/ContainerRollingLogAppender.java
@@ -34,11 +34,12 @@ import java.io.Flushable;
 public class ContainerRollingLogAppender extends RollingFileAppender
   implements Flushable {
   private String containerLogDir;
+  private String containerLogFile;
 
   @Override
   public void activateOptions() {
     synchronized (this) {
-      setFile(new File(this.containerLogDir, "syslog").toString());
+      setFile(new File(this.containerLogDir, containerLogFile).toString());
       setAppend(true);
       super.activateOptions();
     }
@@ -62,4 +63,12 @@ public class ContainerRollingLogAppender extends RollingFileAppender
   public void setContainerLogDir(String containerLogDir) {
     this.containerLogDir = containerLogDir;
   }
+
+  public String getContainerLogFile() {
+    return containerLogFile;
+  }
+
+  public void setContainerLogFile(String containerLogFile) {
+    this.containerLogFile = containerLogFile;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/03ab24aa/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/TestContainerLogAppender.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/TestContainerLogAppender.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/TestContainerLogAppender.java
index 3b20d18..6b8e537 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/TestContainerLogAppender.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/TestContainerLogAppender.java
@@ -30,6 +30,7 @@ public class TestContainerLogAppender {
     claAppender.setName("testCLA");
     claAppender.setLayout(new PatternLayout("%-5p [%t]: %m%n"));
     claAppender.setContainerLogDir("target/testAppendInClose/logDir");
+    claAppender.setContainerLogFile("syslog");
     claAppender.setTotalLogFileSize(1000);
     claAppender.activateOptions();
     final Logger claLog = Logger.getLogger("testAppendInClose-catergory");

http://git-wip-us.apache.org/repos/asf/hadoop/blob/03ab24aa/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/resources/container-log4j.properties
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/resources/container-log4j.properties b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/resources/container-log4j.properties
index 06dec52..cf499b8 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/resources/container-log4j.properties
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/resources/container-log4j.properties
@@ -13,6 +13,7 @@
 #
 # Define some default values that can be overridden by system properties
 hadoop.root.logger=DEBUG,CLA
+yarn.app.mapreduce.shuffle.logger=${hadoop.root.logger}
 
 # Define the root logger to the system property "hadoop.root.logger".
 log4j.rootLogger=${hadoop.root.logger}, EventCounter
@@ -30,18 +31,44 @@ yarn.app.container.log.filesize=100
 
 log4j.appender.CLA=org.apache.hadoop.yarn.ContainerLogAppender
 log4j.appender.CLA.containerLogDir=${yarn.app.container.log.dir}
+log4j.appender.CLA.containerLogFile=${hadoop.root.logfile}
 log4j.appender.CLA.totalLogFileSize=${yarn.app.container.log.filesize}
-
 log4j.appender.CLA.layout=org.apache.log4j.PatternLayout
 log4j.appender.CLA.layout.ConversionPattern=%d{ISO8601} %p [%t] %c: %m%n
 
 log4j.appender.CRLA=org.apache.hadoop.yarn.ContainerRollingLogAppender
 log4j.appender.CRLA.containerLogDir=${yarn.app.container.log.dir}
+log4j.appender.CRLA.containerLogFile=${hadoop.root.logfile}
 log4j.appender.CRLA.maximumFileSize=${yarn.app.container.log.filesize}
 log4j.appender.CRLA.maxBackupIndex=${yarn.app.container.log.backups}
 log4j.appender.CRLA.layout=org.apache.log4j.PatternLayout
 log4j.appender.CRLA.layout.ConversionPattern=%d{ISO8601} %p [%t] %c: %m%n
 
+log4j.appender.shuffleCLA=org.apache.hadoop.yarn.ContainerLogAppender
+log4j.appender.shuffleCLA.containerLogDir=${yarn.app.container.log.dir}
+log4j.appender.shuffleCLA.containerLogFile=${yarn.app.mapreduce.shuffle.logfile}
+log4j.appender.shuffleCLA.totalLogFileSize=${yarn.app.mapreduce.shuffle.log.filesize}
+log4j.appender.shuffleCLA.layout=org.apache.log4j.PatternLayout
+log4j.appender.shuffleCLA.layout.ConversionPattern=%d{ISO8601} %p [%t] %c: %m%n
+
+log4j.appender.shuffleCRLA=org.apache.hadoop.yarn.ContainerRollingLogAppender
+log4j.appender.shuffleCRLA.containerLogDir=${yarn.app.container.log.dir}
+log4j.appender.shuffleCRLA.containerLogFile=${yarn.app.mapreduce.shuffle.logfile}
+log4j.appender.shuffleCRLA.maximumFileSize=${yarn.app.mapreduce.shuffle.log.filesize}
+log4j.appender.shuffleCRLA.maxBackupIndex=${yarn.app.mapreduce.shuffle.log.backups}
+log4j.appender.shuffleCRLA.layout=org.apache.log4j.PatternLayout
+log4j.appender.shuffleCRLA.layout.ConversionPattern=%d{ISO8601} %p [%t] %c: %m%n
+
+################################################################################
+# Shuffle Logger
+#
+log4j.logger.org.apache.hadoop.mapreduce.task.reduce=${yarn.app.mapreduce.shuffle.logger}
+log4j.additivity.org.apache.hadoop.mapreduce.task.reduce=false
+# Merger is used for both map-side and reduce-side spill merging. On the map
+# side yarn.app.mapreduce.shuffle.logger == hadoop.root.logger
+#
+log4j.logger.org.apache.hadoop.mapred.Merger=${yarn.app.mapreduce.shuffle.logger}
+log4j.additivity.org.apache.hadoop.mapred.Merger=false
 #
 # Event Counter Appender
 # Sends counts of logging messages at different severity levels to Hadoop Metrics.


[24/29] hadoop git commit: HADOOP-11343. Overflow is not properly handled in caclulating final iv for AES CTR. Contributed by Jerry Chen.

Posted by vi...@apache.org.
HADOOP-11343. Overflow is not properly handled in caclulating final iv for AES CTR. Contributed by Jerry Chen.


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

Branch: refs/heads/HDFS-EC
Commit: 0707e4eca906552c960e3b8c4e20d9913145eca6
Parents: e69af83
Author: Andrew Wang <wa...@apache.org>
Authored: Fri Dec 5 18:20:19 2014 -0800
Committer: Andrew Wang <wa...@apache.org>
Committed: Fri Dec 5 18:20:19 2014 -0800

----------------------------------------------------------------------
 hadoop-common-project/hadoop-common/CHANGES.txt |  3 +
 .../apache/hadoop/crypto/AesCtrCryptoCodec.java | 27 ++++-----
 .../apache/hadoop/crypto/TestCryptoCodec.java   | 64 ++++++++++++++++++++
 3 files changed, 79 insertions(+), 15 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/0707e4ec/hadoop-common-project/hadoop-common/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/CHANGES.txt b/hadoop-common-project/hadoop-common/CHANGES.txt
index 7a6a938..965c6d3 100644
--- a/hadoop-common-project/hadoop-common/CHANGES.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES.txt
@@ -508,6 +508,9 @@ Release 2.7.0 - UNRELEASED
     HADOOP-11355. When accessing data in HDFS and the key has been deleted,
     a Null Pointer Exception is shown. (Arun Suresh via wang)
 
+    HADOOP-11343. Overflow is not properly handled in caclulating final iv for
+    AES CTR. (Jerry Chen via wang)
+
 Release 2.6.0 - 2014-11-18
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0707e4ec/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/AesCtrCryptoCodec.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/AesCtrCryptoCodec.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/AesCtrCryptoCodec.java
index 8f8bc66..5e286b9 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/AesCtrCryptoCodec.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/AesCtrCryptoCodec.java
@@ -33,7 +33,6 @@ public abstract class AesCtrCryptoCodec extends CryptoCodec {
    * @see http://en.wikipedia.org/wiki/Advanced_Encryption_Standard
    */
   private static final int AES_BLOCK_SIZE = SUITE.getAlgorithmBlockSize();
-  private static final int CTR_OFFSET = 8;
 
   @Override
   public CipherSuite getCipherSuite() {
@@ -48,20 +47,18 @@ public abstract class AesCtrCryptoCodec extends CryptoCodec {
   public void calculateIV(byte[] initIV, long counter, byte[] IV) {
     Preconditions.checkArgument(initIV.length == AES_BLOCK_SIZE);
     Preconditions.checkArgument(IV.length == AES_BLOCK_SIZE);
-    
-    System.arraycopy(initIV, 0, IV, 0, CTR_OFFSET);
-    long l = 0;
-    for (int i = 0; i < 8; i++) {
-      l = ((l << 8) | (initIV[CTR_OFFSET + i] & 0xff));
+
+    int i = IV.length; // IV length
+    int j = 0; // counter bytes index
+    int sum = 0;
+    while (i-- > 0) {
+      // (sum >>> Byte.SIZE) is the carry for addition
+      sum = (initIV[i] & 0xff) + (sum >>> Byte.SIZE);
+      if (j++ < 8) { // Big-endian, and long is 8 bytes length
+        sum += (byte) counter & 0xff;
+        counter >>>= 8;
+      }
+      IV[i] = (byte) sum;
     }
-    l += counter;
-    IV[CTR_OFFSET + 0] = (byte) (l >>> 56);
-    IV[CTR_OFFSET + 1] = (byte) (l >>> 48);
-    IV[CTR_OFFSET + 2] = (byte) (l >>> 40);
-    IV[CTR_OFFSET + 3] = (byte) (l >>> 32);
-    IV[CTR_OFFSET + 4] = (byte) (l >>> 24);
-    IV[CTR_OFFSET + 5] = (byte) (l >>> 16);
-    IV[CTR_OFFSET + 6] = (byte) (l >>> 8);
-    IV[CTR_OFFSET + 7] = (byte) (l);
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0707e4ec/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/crypto/TestCryptoCodec.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/crypto/TestCryptoCodec.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/crypto/TestCryptoCodec.java
index 79987ce..08231f9 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/crypto/TestCryptoCodec.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/crypto/TestCryptoCodec.java
@@ -23,7 +23,9 @@ import static org.junit.Assert.assertTrue;
 import java.io.BufferedInputStream;
 import java.io.DataInputStream;
 import java.io.IOException;
+import java.math.BigInteger;
 import java.security.GeneralSecurityException;
+import java.security.SecureRandom;
 import java.util.Arrays;
 import java.util.HashMap;
 import java.util.Map;
@@ -41,6 +43,8 @@ import org.junit.Assert;
 import org.junit.Assume;
 import org.junit.Test;
 
+import com.google.common.primitives.Longs;
+
 public class TestCryptoCodec {
   private static final Log LOG= LogFactory.getLog(TestCryptoCodec.class);
   private static final byte[] key = {0x01, 0x02, 0x03, 0x04, 0x05, 0x06, 
@@ -230,4 +234,64 @@ public class TestCryptoCodec {
     Assert.assertEquals(len, rand1.length);
     Assert.assertFalse(Arrays.equals(rand, rand1));
   }
+  
+  /**
+   * Regression test for IV calculation, see HADOOP-11343
+   */
+  @Test(timeout=120000)
+  public void testCalculateIV() throws Exception {
+    JceAesCtrCryptoCodec codec = new JceAesCtrCryptoCodec();
+    codec.setConf(conf);
+
+    SecureRandom sr = new SecureRandom();
+    byte[] initIV = new byte[16];
+    byte[] IV = new byte[16];
+
+    long iterations = 1000;
+    long counter = 10000;
+
+    // Overflow test, IV: 00 00 00 00 00 00 00 00 ff ff ff ff ff ff ff ff 
+    for(int i = 0; i < 8; i++) {
+      initIV[8 + i] = (byte)0xff;
+    }
+
+    for(long j = 0; j < counter; j++) {
+      assertIVCalculation(codec, initIV, j, IV);
+    }
+
+    // Random IV and counter sequence test
+    for(long i = 0; i < iterations; i++) {
+      sr.nextBytes(initIV);
+
+      for(long j = 0; j < counter; j++) {
+        assertIVCalculation(codec, initIV, j, IV);
+      }
+    }
+
+    // Random IV and random counter test
+    for(long i = 0; i < iterations; i++) {
+      sr.nextBytes(initIV);
+
+      for(long j = 0; j < counter; j++) {
+        long c = sr.nextLong();
+        assertIVCalculation(codec, initIV, c, IV);
+      }
+    }
+  }
+
+  private void assertIVCalculation(CryptoCodec codec, byte[] initIV,
+      long counter, byte[] IV) {
+    codec.calculateIV(initIV, counter, IV);
+
+    BigInteger iv = new BigInteger(1, IV);
+    BigInteger ref = calculateRef(initIV, counter);
+
+    assertTrue("Calculated IV don't match with the reference", iv.equals(ref));
+  }
+
+  private static BigInteger calculateRef(byte[] initIV, long counter) {
+    byte[] cb = Longs.toByteArray(counter);
+    BigInteger bi = new BigInteger(1, initIV);
+    return bi.add(new BigInteger(1, cb));
+  }
 }


[18/29] hadoop git commit: HDFS-7472. Fix typo in message of ReplicaNotFoundException. Contributed by Masatake Iwasaki.

Posted by vi...@apache.org.
HDFS-7472. Fix typo in message of ReplicaNotFoundException. Contributed by Masatake Iwasaki.


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

Branch: refs/heads/HDFS-EC
Commit: f6452eb2592a9350bc3f6ce1e354ea55b275ff83
Parents: 6a5596e
Author: Haohui Mai <wh...@apache.org>
Authored: Fri Dec 5 11:23:13 2014 -0800
Committer: Haohui Mai <wh...@apache.org>
Committed: Fri Dec 5 11:23:13 2014 -0800

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt                       | 3 +++
 .../hadoop/hdfs/server/datanode/ReplicaNotFoundException.java     | 2 +-
 2 files changed, 4 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/f6452eb2/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index c6cb185..22f462f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -536,6 +536,9 @@ Release 2.7.0 - UNRELEASED
     HDFS-7448 TestBookKeeperHACheckpoints fails in trunk build
     (Akira Ajisaka via stevel)
 
+    HDFS-7472. Fix typo in message of ReplicaNotFoundException.
+    (Masatake Iwasaki via wheat9)
+
 Release 2.6.1 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f6452eb2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ReplicaNotFoundException.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ReplicaNotFoundException.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ReplicaNotFoundException.java
index 124574b..b159d3a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ReplicaNotFoundException.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ReplicaNotFoundException.java
@@ -37,7 +37,7 @@ public class ReplicaNotFoundException extends IOException {
   public final static String NON_EXISTENT_REPLICA =
     "Cannot append to a non-existent replica ";
   public final static String UNEXPECTED_GS_REPLICA =
-    "Cannot append to a replica with unexpeted generation stamp ";
+    "Cannot append to a replica with unexpected generation stamp ";
 
   public ReplicaNotFoundException() {
     super();


[08/29] hadoop git commit: YARN-2880. Added a test to make sure node labels will be recovered if RM restart is enabled. Contributed by Rohith Sharmaks

Posted by vi...@apache.org.
YARN-2880. Added a test to make sure node labels will be recovered if RM restart is enabled. Contributed by Rohith Sharmaks


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

Branch: refs/heads/HDFS-EC
Commit: 73fbb3c66b0d90abee49c766ee9d2f05517cb9de
Parents: a31e016
Author: Jian He <ji...@apache.org>
Authored: Wed Dec 3 17:14:52 2014 -0800
Committer: Jian He <ji...@apache.org>
Committed: Wed Dec 3 17:14:52 2014 -0800

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                 |  3 +
 .../server/resourcemanager/TestRMRestart.java   | 91 ++++++++++++++++++++
 2 files changed, 94 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/73fbb3c6/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index 91151ad..30b9260 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -57,6 +57,9 @@ Release 2.7.0 - UNRELEASED
     YARN-2765. Added leveldb-based implementation for RMStateStore. (Jason Lowe
     via jianhe)
 
+    YARN-2880. Added a test to make sure node labels will be recovered
+    if RM restart is enabled. (Rohith Sharmaks via jianhe)
+
   IMPROVEMENTS
 
     YARN-2891. Failed Container Executor does not provide a clear error

http://git-wip-us.apache.org/repos/asf/hadoop/blob/73fbb3c6/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMRestart.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMRestart.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMRestart.java
index a42170b..29f0208 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMRestart.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMRestart.java
@@ -69,6 +69,7 @@ import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ContainerState;
 import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
+import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.Priority;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.ResourceRequest;
@@ -82,6 +83,7 @@ import org.apache.hadoop.yarn.security.client.RMDelegationTokenIdentifier;
 import org.apache.hadoop.yarn.server.api.protocolrecords.NMContainerStatus;
 import org.apache.hadoop.yarn.server.api.protocolrecords.NodeHeartbeatResponse;
 import org.apache.hadoop.yarn.server.api.records.NodeAction;
+import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager;
 import org.apache.hadoop.yarn.server.resourcemanager.recovery.MemoryRMStateStore;
 import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore;
 import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore.RMState;
@@ -105,6 +107,9 @@ import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
 
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.Sets;
+
 public class TestRMRestart extends ParameterizedSchedulerTestBase {
   private final static File TEMP_DIR = new File(System.getProperty(
     "test.build.data", "/tmp"), "decommision");
@@ -2036,4 +2041,90 @@ public class TestRMRestart extends ParameterizedSchedulerTestBase {
     }
   }
 
+  // Test does following verification
+  // 1. Start RM1 with store patch /tmp
+  // 2. Add/remove/replace labels to cluster and node lable and verify
+  // 3. Start RM2 with store patch /tmp only
+  // 4. Get cluster and node lobel, it should be present by recovering it
+  @Test(timeout = 20000)
+  public void testRMRestartRecoveringNodeLabelManager() throws Exception {
+    MemoryRMStateStore memStore = new MemoryRMStateStore();
+    memStore.init(conf);
+    MockRM rm1 = new MockRM(conf, memStore) {
+      @Override
+      protected RMNodeLabelsManager createNodeLabelManager() {
+        RMNodeLabelsManager mgr = new RMNodeLabelsManager();
+        mgr.init(getConfig());
+        return mgr;
+      }
+    };
+    rm1.init(conf);
+    rm1.start();
+
+    RMNodeLabelsManager nodeLabelManager =
+        rm1.getRMContext().getNodeLabelManager();
+
+    Set<String> clusterNodeLabels = new HashSet<String>();
+    clusterNodeLabels.add("x");
+    clusterNodeLabels.add("y");
+    clusterNodeLabels.add("z");
+    // Add node label x,y,z
+    nodeLabelManager.addToCluserNodeLabels(clusterNodeLabels);
+
+    // Add node Label to Node h1->x
+    NodeId n1 = NodeId.newInstance("h1", 0);
+    nodeLabelManager.addLabelsToNode(ImmutableMap.of(n1, toSet("x")));
+    
+    clusterNodeLabels.remove("z");
+    // Remove cluster label z
+    nodeLabelManager.removeFromClusterNodeLabels(toSet("z"));
+    
+    // Replace nodelabel h1->x,y
+    nodeLabelManager.replaceLabelsOnNode(ImmutableMap.of(n1, toSet("x", "y")));
+
+    // Wait for updating store.It is expected NodeStore update should happen
+    // very fast since it has separate dispatcher. So waiting for max 5 seconds,
+    // which is sufficient time to update NodeStore.
+    int count = 10;
+    while (count-- > 0) {
+      if (nodeLabelManager.getNodeLabels().size() > 0) {
+        break;
+      }
+      Thread.sleep(500);
+    }
+
+    Assert.assertEquals(clusterNodeLabels.size(), nodeLabelManager
+        .getClusterNodeLabels().size());
+
+    Map<NodeId, Set<String>> nodeLabels = nodeLabelManager.getNodeLabels();
+    Assert.assertEquals(1, nodeLabelManager.getNodeLabels().size());
+    Assert.assertTrue(nodeLabels.get(n1).equals(toSet("x", "y")));
+
+    MockRM rm2 = new MockRM(conf, memStore) {
+      @Override
+      protected RMNodeLabelsManager createNodeLabelManager() {
+        RMNodeLabelsManager mgr = new RMNodeLabelsManager();
+        mgr.init(getConfig());
+        return mgr;
+      }
+    };
+    rm2.init(conf);
+    rm2.start();
+
+    nodeLabelManager = rm2.getRMContext().getNodeLabelManager();
+    Assert.assertEquals(clusterNodeLabels.size(), nodeLabelManager
+        .getClusterNodeLabels().size());
+
+    nodeLabels = nodeLabelManager.getNodeLabels();
+    Assert.assertEquals(1, nodeLabelManager.getNodeLabels().size());
+    Assert.assertTrue(nodeLabels.get(n1).equals(toSet("x", "y")));
+    rm1.stop();
+    rm2.stop();
+  }
+
+  private <E> Set<E> toSet(E... elements) {
+    Set<E> set = Sets.newHashSet(elements);
+    return set;
+  }
+
 }


[06/29] hadoop git commit: YARN-2874. Dead lock in DelegationTokenRenewer which blocks RM to execute any further apps. (Naganarasimha G R via kasha)

Posted by vi...@apache.org.
YARN-2874. Dead lock in DelegationTokenRenewer which blocks RM to execute any further apps. (Naganarasimha G R via kasha)


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

Branch: refs/heads/HDFS-EC
Commit: 799353e2c7db5af6e40e3521439b5c8a3c5c6a51
Parents: a1e8225
Author: Karthik Kambatla <ka...@apache.org>
Authored: Wed Dec 3 13:44:41 2014 -0800
Committer: Karthik Kambatla <ka...@apache.org>
Committed: Wed Dec 3 13:44:41 2014 -0800

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                         |  3 +++
 .../security/DelegationTokenRenewer.java                | 12 ++++++------
 2 files changed, 9 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/799353e2/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index 421e5ea..d44f46d 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -175,6 +175,9 @@ Release 2.7.0 - UNRELEASED
     YARN-2894. Fixed a bug regarding application view acl when RM fails over.
     (Rohith Sharmaks via jianhe)
 
+    YARN-2874. Dead lock in "DelegationTokenRenewer" which blocks RM to execute
+    any further apps. (Naganarasimha G R via kasha)
+
 Release 2.6.0 - 2014-11-18
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/799353e2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/security/DelegationTokenRenewer.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/security/DelegationTokenRenewer.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/security/DelegationTokenRenewer.java
index 2dc331e..cca6e8d 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/security/DelegationTokenRenewer.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/security/DelegationTokenRenewer.java
@@ -20,7 +20,6 @@ package org.apache.hadoop.yarn.server.resourcemanager.security;
 
 import java.io.IOException;
 import java.nio.ByteBuffer;
-import java.security.PrivilegedAction;
 import java.security.PrivilegedExceptionAction;
 import java.util.ArrayList;
 import java.util.Collection;
@@ -39,6 +38,7 @@ import java.util.concurrent.LinkedBlockingQueue;
 import java.util.concurrent.ThreadFactory;
 import java.util.concurrent.ThreadPoolExecutor;
 import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.locks.ReadWriteLock;
 import java.util.concurrent.locks.ReentrantReadWriteLock;
 
@@ -445,15 +445,15 @@ public class DelegationTokenRenewer extends AbstractService {
    */
   private class RenewalTimerTask extends TimerTask {
     private DelegationTokenToRenew dttr;
-    private boolean cancelled = false;
+    private AtomicBoolean cancelled = new AtomicBoolean(false);
     
     RenewalTimerTask(DelegationTokenToRenew t) {  
       dttr = t;  
     }
     
     @Override
-    public synchronized void run() {
-      if (cancelled) {
+    public void run() {
+      if (cancelled.get()) {
         return;
       }
 
@@ -475,8 +475,8 @@ public class DelegationTokenRenewer extends AbstractService {
     }
 
     @Override
-    public synchronized boolean cancel() {
-      cancelled = true;
+    public boolean cancel() {
+      cancelled.set(true);
       return super.cancel();
     }
   }


[05/29] hadoop git commit: HDFS-7458. Add description to the nfs ports in core-site.xml used by nfs test to avoid confusion. Contributed by Yongjun Zhang

Posted by vi...@apache.org.
HDFS-7458. Add description to the nfs ports in core-site.xml used by nfs test to avoid confusion. Contributed by Yongjun Zhang


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

Branch: refs/heads/HDFS-EC
Commit: a1e822595c3dc5eadbd5430e57bc4691d09d5e68
Parents: 1812241
Author: Brandon Li <br...@apache.org>
Authored: Wed Dec 3 13:31:26 2014 -0800
Committer: Brandon Li <br...@apache.org>
Committed: Wed Dec 3 13:31:26 2014 -0800

----------------------------------------------------------------------
 .../hadoop-hdfs-nfs/src/test/resources/core-site.xml  | 14 ++++++++++++++
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt           |  3 +++
 2 files changed, 17 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/a1e82259/hadoop-hdfs-project/hadoop-hdfs-nfs/src/test/resources/core-site.xml
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-nfs/src/test/resources/core-site.xml b/hadoop-hdfs-project/hadoop-hdfs-nfs/src/test/resources/core-site.xml
index f90ca03..f400bf2 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-nfs/src/test/resources/core-site.xml
+++ b/hadoop-hdfs-project/hadoop-hdfs-nfs/src/test/resources/core-site.xml
@@ -20,10 +20,24 @@
 <property>
   <name>nfs.server.port</name>
   <value>2079</value>
+  <description>
+    Specify the port number used by Hadoop NFS.
+    Notice that the default value here is different than the default Hadoop nfs
+    port 2049 specified in hdfs-default.xml. 2049 is also the default port for
+    Linux nfs. The setting here allows starting Hadoop nfs for testing even if
+    nfs server (linux or Hadoop) is aready running on he same host.
+  </description>
 </property>
 
 <property>
   <name>nfs.mountd.port</name>
   <value>4272</value>
+  <description>
+    Specify the port number used by Hadoop mount daemon.
+    Notice that the default value here is different than 4242 specified in 
+    hdfs-default.xml. This setting allows starting Hadoop nfs mountd for
+    testing even if the Linux or Hadoop mountd is already running on the
+    same host.
+  </description>
 </property>
 </configuration>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a1e82259/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index 1679a71..a244dab 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -421,6 +421,9 @@ Release 2.7.0 - UNRELEASED
 
     HDFS-6735. A minor optimization to avoid pread() be blocked by read()
     inside the same DFSInputStream (Lars Hofhansl via stack)
+    
+    HDFS-7458. Add description to the nfs ports in core-site.xml used by nfs
+    test to avoid confusion (Yongjun Zhang via brandonli)
 
   OPTIMIZATIONS
 


[16/29] hadoop git commit: HADOOP-11356. Removed deprecated o.a.h.fs.permission.AccessControlException. Contributed by Li Lu.

Posted by vi...@apache.org.
HADOOP-11356. Removed deprecated o.a.h.fs.permission.AccessControlException. Contributed by Li Lu.


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

Branch: refs/heads/HDFS-EC
Commit: 2829b7a96ffe6d2ca5e81689c7957e4e97042f2d
Parents: 0653918
Author: Haohui Mai <wh...@apache.org>
Authored: Fri Dec 5 10:49:43 2014 -0800
Committer: Haohui Mai <wh...@apache.org>
Committed: Fri Dec 5 10:49:43 2014 -0800

----------------------------------------------------------------------
 hadoop-common-project/hadoop-common/CHANGES.txt |  3 +
 .../fs/permission/AccessControlException.java   | 66 --------------------
 .../hadoop/security/AccessControlException.java |  4 +-
 3 files changed, 5 insertions(+), 68 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/2829b7a9/hadoop-common-project/hadoop-common/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/CHANGES.txt b/hadoop-common-project/hadoop-common/CHANGES.txt
index f2a086e..2f88fc8 100644
--- a/hadoop-common-project/hadoop-common/CHANGES.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES.txt
@@ -410,6 +410,9 @@ Release 2.7.0 - UNRELEASED
     HADOOP-11301. [optionally] update jmx cache to drop old metrics
     (Maysam Yabandeh via stack)
 
+    HADOOP-11356. Removed deprecated o.a.h.fs.permission.AccessControlException.
+    (Li Lu via wheat9)
+
   OPTIMIZATIONS
 
     HADOOP-11323. WritableComparator#compare keeps reference to byte array.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2829b7a9/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/permission/AccessControlException.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/permission/AccessControlException.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/permission/AccessControlException.java
deleted file mode 100644
index 1cd6395..0000000
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/permission/AccessControlException.java
+++ /dev/null
@@ -1,66 +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.fs.permission;
-
-import java.io.IOException;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-
-/**
- * An exception class for access control related issues.
- * @deprecated Use {@link org.apache.hadoop.security.AccessControlException} 
- *             instead.
- */
-@Deprecated
-@InterfaceAudience.Public
-@InterfaceStability.Stable
-public class AccessControlException extends IOException {
-  //Required by {@link java.io.Serializable}.
-  private static final long serialVersionUID = 1L;
-
-  /**
-   * Default constructor is needed for unwrapping from 
-   * {@link org.apache.hadoop.ipc.RemoteException}.
-   */
-  public AccessControlException() {
-    super("Permission denied.");
-  }
-
-  /**
-   * Constructs an {@link AccessControlException}
-   * with the specified detail message.
-   * @param s the detail message.
-   */
-  public AccessControlException(String s) {
-    super(s);
-  }
-  
-  /**
-   * Constructs a new exception with the specified cause and a detail
-   * message of <tt>(cause==null ? null : cause.toString())</tt> (which
-   * typically contains the class and detail message of <tt>cause</tt>).
-   * @param  cause the cause (which is saved for later retrieval by the
-   *         {@link #getCause()} method).  (A <tt>null</tt> value is
-   *         permitted, and indicates that the cause is nonexistent or
-   *         unknown.)
-   */
-  public AccessControlException(Throwable cause) {
-    super(cause);
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2829b7a9/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/AccessControlException.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/AccessControlException.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/AccessControlException.java
index 5f0faa6..b1ab829 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/AccessControlException.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/AccessControlException.java
@@ -17,6 +17,7 @@
  */
 package org.apache.hadoop.security;
 
+import java.io.IOException;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 
@@ -25,8 +26,7 @@ import org.apache.hadoop.classification.InterfaceStability;
  */
 @InterfaceAudience.LimitedPrivate({"HDFS", "MapReduce"})
 @InterfaceStability.Evolving
-public class AccessControlException 
-    extends org.apache.hadoop.fs.permission.AccessControlException {
+public class AccessControlException extends IOException {
 
   //Required by {@link java.io.Serializable}.
   private static final long serialVersionUID = 1L;


[04/29] hadoop git commit: HADOOP-11342. KMS key ACL should ignore ALL operation for default key ACL and whitelist key ACL. Contributed by Dian Fu.

Posted by vi...@apache.org.
HADOOP-11342. KMS key ACL should ignore ALL operation for default key ACL and whitelist key ACL. Contributed by Dian Fu.


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

Branch: refs/heads/HDFS-EC
Commit: 1812241ee10c0a98844bffb9341f770d54655f52
Parents: 03ab24a
Author: Andrew Wang <wa...@apache.org>
Authored: Wed Dec 3 12:00:14 2014 -0800
Committer: Andrew Wang <wa...@apache.org>
Committed: Wed Dec 3 12:00:14 2014 -0800

----------------------------------------------------------------------
 hadoop-common-project/hadoop-common/CHANGES.txt |  3 +++
 .../hadoop/crypto/key/kms/server/KMSACLs.java   | 26 ++++++++++++++------
 .../hadoop/crypto/key/kms/server/TestKMS.java   |  5 +++-
 3 files changed, 26 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/1812241e/hadoop-common-project/hadoop-common/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/CHANGES.txt b/hadoop-common-project/hadoop-common/CHANGES.txt
index 2f17f22..7a2159f 100644
--- a/hadoop-common-project/hadoop-common/CHANGES.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES.txt
@@ -493,6 +493,9 @@ Release 2.7.0 - UNRELEASED
     HADOOP-11344. KMS kms-config.sh sets a default value for the keystore
     password even in non-ssl setup. (Arun Suresh via wang)
 
+    HADOOP-11342. KMS key ACL should ignore ALL operation for default key ACL
+    and whitelist key ACL. (Dian Fu via wang)
+
 Release 2.6.0 - 2014-11-18
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1812241e/hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KMSACLs.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KMSACLs.java b/hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KMSACLs.java
index 0217589..c33dd4b 100644
--- a/hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KMSACLs.java
+++ b/hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KMSACLs.java
@@ -152,20 +152,30 @@ public class KMSACLs implements Runnable, KeyACLs {
         String confKey = KMSConfiguration.DEFAULT_KEY_ACL_PREFIX + keyOp;
         String aclStr = conf.get(confKey);
         if (aclStr != null) {
-          if (aclStr.equals("*")) {
-            LOG.info("Default Key ACL for KEY_OP '{}' is set to '*'", keyOp);
+          if (keyOp == KeyOpType.ALL) {
+            // Ignore All operation for default key acl
+            LOG.warn("Should not configure default key ACL for KEY_OP '{}'", keyOp);
+          } else {
+            if (aclStr.equals("*")) {
+              LOG.info("Default Key ACL for KEY_OP '{}' is set to '*'", keyOp);
+            }
+            defaultKeyAcls.put(keyOp, new AccessControlList(aclStr));
           }
-          defaultKeyAcls.put(keyOp, new AccessControlList(aclStr));
         }
       }
       if (!whitelistKeyAcls.containsKey(keyOp)) {
         String confKey = KMSConfiguration.WHITELIST_KEY_ACL_PREFIX + keyOp;
         String aclStr = conf.get(confKey);
         if (aclStr != null) {
-          if (aclStr.equals("*")) {
-            LOG.info("Whitelist Key ACL for KEY_OP '{}' is set to '*'", keyOp);
+          if (keyOp == KeyOpType.ALL) {
+            // Ignore All operation for whitelist key acl
+            LOG.warn("Should not configure whitelist key ACL for KEY_OP '{}'", keyOp);
+          } else {
+            if (aclStr.equals("*")) {
+              LOG.info("Whitelist Key ACL for KEY_OP '{}' is set to '*'", keyOp);
+            }
+            whitelistKeyAcls.put(keyOp, new AccessControlList(aclStr));
           }
-          whitelistKeyAcls.put(keyOp, new AccessControlList(aclStr));
         }
       }
     }
@@ -271,7 +281,9 @@ public class KMSACLs implements Runnable, KeyACLs {
 
   @Override
   public boolean isACLPresent(String keyName, KeyOpType opType) {
-    return (keyAcls.containsKey(keyName) || defaultKeyAcls.containsKey(opType));
+    return (keyAcls.containsKey(keyName)
+        || defaultKeyAcls.containsKey(opType)
+        || whitelistKeyAcls.containsKey(opType));
   }
 
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1812241e/hadoop-common-project/hadoop-kms/src/test/java/org/apache/hadoop/crypto/key/kms/server/TestKMS.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-kms/src/test/java/org/apache/hadoop/crypto/key/kms/server/TestKMS.java b/hadoop-common-project/hadoop-kms/src/test/java/org/apache/hadoop/crypto/key/kms/server/TestKMS.java
index d840646..b9409ca 100644
--- a/hadoop-common-project/hadoop-kms/src/test/java/org/apache/hadoop/crypto/key/kms/server/TestKMS.java
+++ b/hadoop-common-project/hadoop-kms/src/test/java/org/apache/hadoop/crypto/key/kms/server/TestKMS.java
@@ -619,16 +619,19 @@ public class TestKMS {
     }
     conf.set(KMSACLs.Type.CREATE.getAclConfigKey(),"CREATE,ROLLOVER,GET,SET_KEY_MATERIAL,GENERATE_EEK,DECRYPT_EEK");
     conf.set(KMSACLs.Type.ROLLOVER.getAclConfigKey(),"CREATE,ROLLOVER,GET,SET_KEY_MATERIAL,GENERATE_EEK,DECRYPT_EEK");
-    conf.set(KMSACLs.Type.GENERATE_EEK.getAclConfigKey(),"CREATE,ROLLOVER,GET,SET_KEY_MATERIAL,GENERATE_EEK");
+    conf.set(KMSACLs.Type.GENERATE_EEK.getAclConfigKey(),"CREATE,ROLLOVER,GET,SET_KEY_MATERIAL,GENERATE_EEK,DECRYPT_EEK");
     conf.set(KMSACLs.Type.DECRYPT_EEK.getAclConfigKey(),"CREATE,ROLLOVER,GET,SET_KEY_MATERIAL,GENERATE_EEK");
 
     conf.set(KeyAuthorizationKeyProvider.KEY_ACL + "test_key.MANAGEMENT", "CREATE");
     conf.set(KeyAuthorizationKeyProvider.KEY_ACL + "some_key.MANAGEMENT", "ROLLOVER");
     conf.set(KMSConfiguration.WHITELIST_KEY_ACL_PREFIX + "MANAGEMENT", "DECRYPT_EEK");
+    conf.set(KMSConfiguration.WHITELIST_KEY_ACL_PREFIX + "ALL", "DECRYPT_EEK");
 
     conf.set(KeyAuthorizationKeyProvider.KEY_ACL + "all_access.ALL", "GENERATE_EEK");
     conf.set(KeyAuthorizationKeyProvider.KEY_ACL + "all_access.DECRYPT_EEK", "ROLLOVER");
     conf.set(KMSConfiguration.DEFAULT_KEY_ACL_PREFIX + "MANAGEMENT", "ROLLOVER");
+    conf.set(KMSConfiguration.DEFAULT_KEY_ACL_PREFIX + "GENERATE_EEK", "SOMEBODY");
+    conf.set(KMSConfiguration.DEFAULT_KEY_ACL_PREFIX + "ALL", "ROLLOVER");
 
     writeConf(testDir, conf);
 


[25/29] hadoop git commit: HDFS-7459. Consolidate cache-related implementation in FSNamesystem into a single class. Contributed by Haohui Mai.

Posted by vi...@apache.org.
HDFS-7459. Consolidate cache-related implementation in FSNamesystem into a single class. Contributed by Haohui Mai.


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

Branch: refs/heads/HDFS-EC
Commit: e227fb8fbcd414717faded9454b8ef813f7aafea
Parents: 0707e4e
Author: Haohui Mai <wh...@apache.org>
Authored: Fri Dec 5 18:35:45 2014 -0800
Committer: Haohui Mai <wh...@apache.org>
Committed: Fri Dec 5 18:37:07 2014 -0800

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |   3 +
 .../hdfs/server/namenode/FSNDNCacheOp.java      | 124 ++++++++++++++++
 .../hdfs/server/namenode/FSNamesystem.java      | 140 ++++++-------------
 3 files changed, 173 insertions(+), 94 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/e227fb8f/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index d4db732..87b02c4 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -435,6 +435,9 @@ Release 2.7.0 - UNRELEASED
 
     HDFS-7474. Avoid resolving path in FSPermissionChecker. (jing9)
 
+    HDFS-7459. Consolidate cache-related implementation in FSNamesystem into
+    a single class. (wheat9)
+
   OPTIMIZATIONS
 
     HDFS-7454. Reduce memory footprint for AclEntries in NameNode.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e227fb8f/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNDNCacheOp.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNDNCacheOp.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNDNCacheOp.java
new file mode 100644
index 0000000..093ee74
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNDNCacheOp.java
@@ -0,0 +1,124 @@
+/**
+ * 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.fs.BatchedRemoteIterator.BatchedListEntries;
+import org.apache.hadoop.fs.CacheFlag;
+import org.apache.hadoop.hdfs.protocol.CacheDirectiveEntry;
+import org.apache.hadoop.hdfs.protocol.CacheDirectiveInfo;
+import org.apache.hadoop.hdfs.protocol.CachePoolEntry;
+import org.apache.hadoop.hdfs.protocol.CachePoolInfo;
+import org.apache.hadoop.security.AccessControlException;
+
+import java.io.IOException;
+import java.util.EnumSet;
+
+class FSNDNCacheOp {
+  static CacheDirectiveInfo addCacheDirective(
+      FSNamesystem fsn, CacheManager cacheManager,
+      CacheDirectiveInfo directive, EnumSet<CacheFlag> flags,
+      boolean logRetryCache)
+      throws IOException {
+
+    final FSPermissionChecker pc = getFsPermissionChecker(fsn);
+
+    if (directive.getId() != null) {
+      throw new IOException("addDirective: you cannot specify an ID " +
+          "for this operation.");
+    }
+    CacheDirectiveInfo effectiveDirective =
+        cacheManager.addDirective(directive, pc, flags);
+    fsn.getEditLog().logAddCacheDirectiveInfo(effectiveDirective,
+        logRetryCache);
+    return effectiveDirective;
+  }
+
+  static void modifyCacheDirective(
+      FSNamesystem fsn, CacheManager cacheManager, CacheDirectiveInfo directive,
+      EnumSet<CacheFlag> flags, boolean logRetryCache) throws IOException {
+    final FSPermissionChecker pc = getFsPermissionChecker(fsn);
+
+    cacheManager.modifyDirective(directive, pc, flags);
+    fsn.getEditLog().logModifyCacheDirectiveInfo(directive, logRetryCache);
+  }
+
+  static void removeCacheDirective(
+      FSNamesystem fsn, CacheManager cacheManager, long id,
+      boolean logRetryCache)
+      throws IOException {
+    final FSPermissionChecker pc = getFsPermissionChecker(fsn);
+
+    cacheManager.removeDirective(id, pc);
+    fsn.getEditLog().logRemoveCacheDirectiveInfo(id, logRetryCache);
+  }
+
+  static BatchedListEntries<CacheDirectiveEntry> listCacheDirectives(
+      FSNamesystem fsn, CacheManager cacheManager,
+      long startId, CacheDirectiveInfo filter) throws IOException {
+    final FSPermissionChecker pc = getFsPermissionChecker(fsn);
+    return cacheManager.listCacheDirectives(startId, filter, pc);
+  }
+
+  static CachePoolInfo addCachePool(
+      FSNamesystem fsn, CacheManager cacheManager, CachePoolInfo req,
+      boolean logRetryCache)
+      throws IOException {
+    final FSPermissionChecker pc = getFsPermissionChecker(fsn);
+
+    if (pc != null) {
+      pc.checkSuperuserPrivilege();
+    }
+    CachePoolInfo info = cacheManager.addCachePool(req);
+    fsn.getEditLog().logAddCachePool(info, logRetryCache);
+    return info;
+  }
+
+  static void modifyCachePool(
+      FSNamesystem fsn, CacheManager cacheManager, CachePoolInfo req,
+      boolean logRetryCache) throws IOException {
+    final FSPermissionChecker pc = getFsPermissionChecker(fsn);
+    if (pc != null) {
+      pc.checkSuperuserPrivilege();
+    }
+    cacheManager.modifyCachePool(req);
+    fsn.getEditLog().logModifyCachePool(req, logRetryCache);
+  }
+
+  static void removeCachePool(
+      FSNamesystem fsn, CacheManager cacheManager, String cachePoolName,
+      boolean logRetryCache) throws IOException {
+    final FSPermissionChecker pc = getFsPermissionChecker(fsn);
+    if (pc != null) {
+      pc.checkSuperuserPrivilege();
+    }
+    cacheManager.removeCachePool(cachePoolName);
+    fsn.getEditLog().logRemoveCachePool(cachePoolName, logRetryCache);
+  }
+
+  static BatchedListEntries<CachePoolEntry> listCachePools(
+      FSNamesystem fsn, CacheManager cacheManager, String prevKey)
+      throws IOException {
+    final FSPermissionChecker pc = getFsPermissionChecker(fsn);
+    return cacheManager.listCachePools(pc, prevKey);
+  }
+
+  private static FSPermissionChecker getFsPermissionChecker(FSNamesystem fsn)
+      throws AccessControlException {
+    return fsn.isPermissionEnabled() ? fsn.getPermissionChecker() : null;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e227fb8f/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 0a558dd..982798f 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
@@ -1354,7 +1354,11 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
       }
     }
   }
-  
+
+  boolean isPermissionEnabled() {
+    return isPermissionEnabled;
+  }
+
   /**
    * We already know that the safemode is on. We will throw a RetriableException
    * if the safemode is not manual or caused by low resource.
@@ -3607,7 +3611,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     return status;
   }
     
-  private FSPermissionChecker getPermissionChecker()
+  FSPermissionChecker getPermissionChecker()
       throws AccessControlException {
     return dir.getPermissionChecker();
   }
@@ -7541,52 +7545,38 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
                          EnumSet<CacheFlag> flags, boolean logRetryCache)
       throws IOException {
     checkOperation(OperationCategory.WRITE);
-    final FSPermissionChecker pc = isPermissionEnabled ?
-        getPermissionChecker() : null;
-
+    CacheDirectiveInfo effectiveDirective = null;
     if (!flags.contains(CacheFlag.FORCE)) {
       cacheManager.waitForRescanIfNeeded();
     }
-    boolean success = false;
     writeLock();
-    String effectiveDirectiveStr = null;
-    Long result = null;
     try {
       checkOperation(OperationCategory.WRITE);
       if (isInSafeMode()) {
         throw new SafeModeException(
             "Cannot add cache directive", safeMode);
       }
-      if (directive.getId() != null) {
-        throw new IOException("addDirective: you cannot specify an ID " +
-            "for this operation.");
-      }
-      CacheDirectiveInfo effectiveDirective =
-          cacheManager.addDirective(directive, pc, flags);
-      getEditLog().logAddCacheDirectiveInfo(effectiveDirective, logRetryCache);
-      result = effectiveDirective.getId();
-      effectiveDirectiveStr = effectiveDirective.toString();
-      success = true;
+      effectiveDirective = FSNDNCacheOp.addCacheDirective(this, cacheManager,
+          directive, flags, logRetryCache);
     } finally {
       writeUnlock();
+      boolean success = effectiveDirective != null;
       if (success) {
         getEditLog().logSync();
       }
-      if (isAuditEnabled() && isExternalInvocation()) {
-        logAuditEvent(success, "addCacheDirective", effectiveDirectiveStr, null, null);
-      }
 
+      String effectiveDirectiveStr = effectiveDirective != null ?
+          effectiveDirective.toString() : null;
+      logAuditEvent(success, "addCacheDirective", effectiveDirectiveStr,
+          null, null);
     }
-    return result;
+    return effectiveDirective != null ? effectiveDirective.getId() : 0;
   }
 
   void modifyCacheDirective(CacheDirectiveInfo directive,
       EnumSet<CacheFlag> flags, boolean logRetryCache) throws IOException {
     checkOperation(OperationCategory.WRITE);
-    final FSPermissionChecker pc = isPermissionEnabled ?
-        getPermissionChecker() : null;
     boolean success = false;
-
     if (!flags.contains(CacheFlag.FORCE)) {
       cacheManager.waitForRescanIfNeeded();
     }
@@ -7597,26 +7587,22 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
         throw new SafeModeException(
             "Cannot add cache directive", safeMode);
       }
-      cacheManager.modifyDirective(directive, pc, flags);
-      getEditLog().logModifyCacheDirectiveInfo(directive, logRetryCache);
+      FSNDNCacheOp.modifyCacheDirective(this, cacheManager, directive, flags,
+          logRetryCache);
       success = true;
     } finally {
       writeUnlock();
       if (success) {
         getEditLog().logSync();
       }
-      if (isAuditEnabled() && isExternalInvocation()) {
-        String idStr = "{id: " + directive.getId().toString() + "}";
-        logAuditEvent(success, "modifyCacheDirective", idStr, directive.toString(), null);
-      }
+      String idStr = "{id: " + directive.getId().toString() + "}";
+      logAuditEvent(success, "modifyCacheDirective", idStr,
+          directive.toString(), null);
     }
   }
 
   void removeCacheDirective(long id, boolean logRetryCache) throws IOException {
     checkOperation(OperationCategory.WRITE);
-    final FSPermissionChecker pc = isPermissionEnabled ?
-        getPermissionChecker() : null;
-
     boolean success = false;
     writeLock();
     try {
@@ -7625,16 +7611,13 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
         throw new SafeModeException(
             "Cannot remove cache directives", safeMode);
       }
-      cacheManager.removeDirective(id, pc);
-      getEditLog().logRemoveCacheDirectiveInfo(id, logRetryCache);
+      FSNDNCacheOp.removeCacheDirective(this, cacheManager, id, logRetryCache);
       success = true;
     } finally {
       writeUnlock();
-      if (isAuditEnabled() && isExternalInvocation()) {
-        String idStr = "{id: " + Long.toString(id) + "}";
-        logAuditEvent(success, "removeCacheDirective", idStr, null,
-            null);
-      }
+      String idStr = "{id: " + Long.toString(id) + "}";
+      logAuditEvent(success, "removeCacheDirective", idStr, null,
+          null);
     }
     getEditLog().logSync();
   }
@@ -7642,33 +7625,26 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
   BatchedListEntries<CacheDirectiveEntry> listCacheDirectives(
       long startId, CacheDirectiveInfo filter) throws IOException {
     checkOperation(OperationCategory.READ);
-    final FSPermissionChecker pc = isPermissionEnabled ?
-        getPermissionChecker() : null;
     BatchedListEntries<CacheDirectiveEntry> results;
     cacheManager.waitForRescanIfNeeded();
     readLock();
     boolean success = false;
     try {
       checkOperation(OperationCategory.READ);
-      results =
-          cacheManager.listCacheDirectives(startId, filter, pc);
+      results = FSNDNCacheOp.listCacheDirectives(this, cacheManager, startId,
+          filter);
       success = true;
     } finally {
       readUnlock();
-      if (isAuditEnabled() && isExternalInvocation()) {
-        logAuditEvent(success, "listCacheDirectives", filter.toString(), null,
-            null);
-      }
+      logAuditEvent(success, "listCacheDirectives", filter.toString(), null,
+          null);
     }
     return results;
   }
 
-  public void addCachePool(CachePoolInfo req, boolean logRetryCache)
+  void addCachePool(CachePoolInfo req, boolean logRetryCache)
       throws IOException {
     checkOperation(OperationCategory.WRITE);
-    final FSPermissionChecker pc = isPermissionEnabled ?
-        getPermissionChecker() : null;
-
     writeLock();
     boolean success = false;
     String poolInfoStr = null;
@@ -7678,29 +7654,21 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
         throw new SafeModeException(
             "Cannot add cache pool " + req.getPoolName(), safeMode);
       }
-      if (pc != null) {
-        pc.checkSuperuserPrivilege();
-      }
-      CachePoolInfo info = cacheManager.addCachePool(req);
+      CachePoolInfo info = FSNDNCacheOp.addCachePool(this, cacheManager, req,
+          logRetryCache);
       poolInfoStr = info.toString();
-      getEditLog().logAddCachePool(info, logRetryCache);
       success = true;
     } finally {
       writeUnlock();
-      if (isAuditEnabled() && isExternalInvocation()) {
-        logAuditEvent(success, "addCachePool", poolInfoStr, null, null);
-      }
+      logAuditEvent(success, "addCachePool", poolInfoStr, null, null);
     }
     
     getEditLog().logSync();
   }
 
-  public void modifyCachePool(CachePoolInfo req, boolean logRetryCache)
+  void modifyCachePool(CachePoolInfo req, boolean logRetryCache)
       throws IOException {
     checkOperation(OperationCategory.WRITE);
-    final FSPermissionChecker pc =
-        isPermissionEnabled ? getPermissionChecker() : null;
-
     writeLock();
     boolean success = false;
     try {
@@ -7709,29 +7677,22 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
         throw new SafeModeException(
             "Cannot modify cache pool " + req.getPoolName(), safeMode);
       }
-      if (pc != null) {
-        pc.checkSuperuserPrivilege();
-      }
-      cacheManager.modifyCachePool(req);
-      getEditLog().logModifyCachePool(req, logRetryCache);
+      FSNDNCacheOp.modifyCachePool(this, cacheManager, req, logRetryCache);
       success = true;
     } finally {
       writeUnlock();
-      if (isAuditEnabled() && isExternalInvocation()) {
-        String poolNameStr = "{poolName: " + req.getPoolName() + "}";
-        logAuditEvent(success, "modifyCachePool", poolNameStr, req.toString(), null);
-      }
+      String poolNameStr = "{poolName: " +
+          (req == null ? null : req.getPoolName()) + "}";
+      logAuditEvent(success, "modifyCachePool", poolNameStr,
+                    req == null ? null : req.toString(), null);
     }
 
     getEditLog().logSync();
   }
 
-  public void removeCachePool(String cachePoolName, boolean logRetryCache)
+  void removeCachePool(String cachePoolName, boolean logRetryCache)
       throws IOException {
     checkOperation(OperationCategory.WRITE);
-    final FSPermissionChecker pc =
-        isPermissionEnabled ? getPermissionChecker() : null;
-
     writeLock();
     boolean success = false;
     try {
@@ -7740,27 +7701,20 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
         throw new SafeModeException(
             "Cannot remove cache pool " + cachePoolName, safeMode);
       }
-      if (pc != null) {
-        pc.checkSuperuserPrivilege();
-      }
-      cacheManager.removeCachePool(cachePoolName);
-      getEditLog().logRemoveCachePool(cachePoolName, logRetryCache);
+      FSNDNCacheOp.removeCachePool(this, cacheManager, cachePoolName,
+          logRetryCache);
       success = true;
     } finally {
       writeUnlock();
-      if (isAuditEnabled() && isExternalInvocation()) {
-        String poolNameStr = "{poolName: " + cachePoolName + "}";
-        logAuditEvent(success, "removeCachePool", poolNameStr, null, null);
-      }
+      String poolNameStr = "{poolName: " + cachePoolName + "}";
+      logAuditEvent(success, "removeCachePool", poolNameStr, null, null);
     }
     
     getEditLog().logSync();
   }
 
-  public BatchedListEntries<CachePoolEntry> listCachePools(String prevKey)
+  BatchedListEntries<CachePoolEntry> listCachePools(String prevKey)
       throws IOException {
-    final FSPermissionChecker pc =
-        isPermissionEnabled ? getPermissionChecker() : null;
     BatchedListEntries<CachePoolEntry> results;
     checkOperation(OperationCategory.READ);
     boolean success = false;
@@ -7768,13 +7722,11 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     readLock();
     try {
       checkOperation(OperationCategory.READ);
-      results = cacheManager.listCachePools(pc, prevKey);
+      results = FSNDNCacheOp.listCachePools(this, cacheManager, prevKey);
       success = true;
     } finally {
       readUnlock();
-      if (isAuditEnabled() && isExternalInvocation()) {
-        logAuditEvent(success, "listCachePools", null, null, null);
-      }
+      logAuditEvent(success, "listCachePools", null, null, null);
     }
     return results;
   }


[02/29] hadoop git commit: HDFS-7448 TestBookKeeperHACheckpoints fails in trunk -move CHANGES.TXT entry

Posted by vi...@apache.org.
HDFS-7448 TestBookKeeperHACheckpoints fails in trunk -move CHANGES.TXT entry


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

Branch: refs/heads/HDFS-EC
Commit: 22afae890d7cf34a9be84590e7457774755b7a4a
Parents: e65b7c5
Author: Steve Loughran <st...@apache.org>
Authored: Wed Dec 3 12:21:42 2014 +0000
Committer: Steve Loughran <st...@apache.org>
Committed: Wed Dec 3 12:21:42 2014 +0000

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


http://git-wip-us.apache.org/repos/asf/hadoop/blob/22afae89/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index 85d00b7..1679a71 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -257,9 +257,6 @@ Trunk (Unreleased)
 
     HDFS-7407. Minor typo in privileged pid/out/log names (aw)
 
-    HDFS-7448 TestBookKeeperHACheckpoints fails in trunk build
-    (Akira Ajisaka via stevel)
-
 Release 2.7.0 - UNRELEASED
 
   INCOMPATIBLE CHANGES
@@ -522,6 +519,9 @@ Release 2.7.0 - UNRELEASED
     HDFS-7444. convertToBlockUnderConstruction should preserve BlockCollection.
     (wheat9)
 
+    HDFS-7448 TestBookKeeperHACheckpoints fails in trunk build
+    (Akira Ajisaka via stevel)
+
 Release 2.6.1 - UNRELEASED
 
   INCOMPATIBLE CHANGES


[09/29] hadoop git commit: HADOOP-11332. KerberosAuthenticator#doSpnegoSequence should check if kerberos TGT is available in the subject. Contributed by Dian Fu.

Posted by vi...@apache.org.
HADOOP-11332. KerberosAuthenticator#doSpnegoSequence should check if kerberos TGT is available in the subject. Contributed by Dian Fu.


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

Branch: refs/heads/HDFS-EC
Commit: 9d1a8f5897d585bec96de32116fbd2118f8e0f95
Parents: 73fbb3c
Author: Aaron T. Myers <at...@apache.org>
Authored: Wed Dec 3 18:53:45 2014 -0800
Committer: Aaron T. Myers <at...@apache.org>
Committed: Wed Dec 3 18:53:45 2014 -0800

----------------------------------------------------------------------
 .../security/authentication/client/KerberosAuthenticator.java  | 6 +++++-
 hadoop-common-project/hadoop-common/CHANGES.txt                | 3 +++
 2 files changed, 8 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/9d1a8f58/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/client/KerberosAuthenticator.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/client/KerberosAuthenticator.java b/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/client/KerberosAuthenticator.java
index e4ebf1b..928866c 100644
--- a/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/client/KerberosAuthenticator.java
+++ b/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/client/KerberosAuthenticator.java
@@ -23,6 +23,8 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import javax.security.auth.Subject;
+import javax.security.auth.kerberos.KerberosKey;
+import javax.security.auth.kerberos.KerberosTicket;
 import javax.security.auth.login.AppConfigurationEntry;
 import javax.security.auth.login.Configuration;
 import javax.security.auth.login.LoginContext;
@@ -247,7 +249,9 @@ public class KerberosAuthenticator implements Authenticator {
     try {
       AccessControlContext context = AccessController.getContext();
       Subject subject = Subject.getSubject(context);
-      if (subject == null) {
+      if (subject == null
+          || (subject.getPrivateCredentials(KerberosKey.class).isEmpty()
+              && subject.getPrivateCredentials(KerberosTicket.class).isEmpty())) {
         LOG.debug("No subject in context, logging in");
         subject = new Subject();
         LoginContext login = new LoginContext("", subject,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9d1a8f58/hadoop-common-project/hadoop-common/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/CHANGES.txt b/hadoop-common-project/hadoop-common/CHANGES.txt
index 7a2159f..f53bceb 100644
--- a/hadoop-common-project/hadoop-common/CHANGES.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES.txt
@@ -496,6 +496,9 @@ Release 2.7.0 - UNRELEASED
     HADOOP-11342. KMS key ACL should ignore ALL operation for default key ACL
     and whitelist key ACL. (Dian Fu via wang)
 
+    HADOOP-11332. KerberosAuthenticator#doSpnegoSequence should check if
+    kerberos TGT is available in the subject. (Dian Fu via atm)
+
 Release 2.6.0 - 2014-11-18
 
   INCOMPATIBLE CHANGES


[14/29] hadoop git commit: YARN-2189. [YARN-1492] Admin service for cache manager. (Chris Trezzo via kasha)

Posted by vi...@apache.org.
YARN-2189. [YARN-1492] Admin service for cache manager. (Chris Trezzo via kasha)


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

Branch: refs/heads/HDFS-EC
Commit: 78968155d7f87f2147faf96c5eef9c23dba38db8
Parents: 26d8dec
Author: Karthik Kambatla <ka...@apache.org>
Authored: Thu Dec 4 17:36:32 2014 -0800
Committer: Karthik Kambatla <ka...@apache.org>
Committed: Thu Dec 4 17:36:32 2014 -0800

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                 |   3 +
 hadoop-yarn-project/hadoop-yarn/bin/yarn        |   5 +
 .../hadoop-yarn/hadoop-yarn-api/pom.xml         |   1 +
 .../hadoop/yarn/conf/YarnConfiguration.java     |  12 ++
 .../yarn/server/api/SCMAdminProtocol.java       |  53 ++++++
 .../yarn/server/api/SCMAdminProtocolPB.java     |  31 ++++
 .../RunSharedCacheCleanerTaskRequest.java       |  37 ++++
 .../RunSharedCacheCleanerTaskResponse.java      |  58 ++++++
 .../main/proto/server/SCM_Admin_protocol.proto  |  29 +++
 .../src/main/proto/yarn_service_protos.proto    |  11 ++
 .../org/apache/hadoop/yarn/client/SCMAdmin.java | 183 +++++++++++++++++++
 .../pb/client/SCMAdminProtocolPBClientImpl.java |  73 ++++++++
 .../service/SCMAdminProtocolPBServiceImpl.java  |  57 ++++++
 .../RunSharedCacheCleanerTaskRequestPBImpl.java |  53 ++++++
 ...RunSharedCacheCleanerTaskResponsePBImpl.java |  66 +++++++
 .../src/main/resources/yarn-default.xml         |  12 ++
 .../SCMAdminProtocolService.java                | 146 +++++++++++++++
 .../sharedcachemanager/SharedCacheManager.java  |   8 +
 .../TestSCMAdminProtocolService.java            | 135 ++++++++++++++
 19 files changed, 973 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/78968155/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index f032b4f..252b7d5 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -54,6 +54,9 @@ Release 2.7.0 - UNRELEASED
     YARN-2188. [YARN-1492] Client service for cache manager. 
     (Chris Trezzo and Sangjin Lee via kasha)
 
+    YARN-2189. [YARN-1492] Admin service for cache manager.
+    (Chris Trezzo via kasha)
+
     YARN-2765. Added leveldb-based implementation for RMStateStore. (Jason Lowe
     via jianhe)
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/78968155/hadoop-yarn-project/hadoop-yarn/bin/yarn
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/bin/yarn b/hadoop-yarn-project/hadoop-yarn/bin/yarn
index b98f344..dfa27e4 100644
--- a/hadoop-yarn-project/hadoop-yarn/bin/yarn
+++ b/hadoop-yarn-project/hadoop-yarn/bin/yarn
@@ -36,6 +36,7 @@ function hadoop_usage
   echo "  resourcemanager -format-state-store   deletes the RMStateStore"
   echo "  rmadmin                               admin tools"
   echo "  sharedcachemanager                    run the SharedCacheManager daemon"
+  echo "  scmadmin                              SharedCacheManager admin tools"
   echo "  timelineserver                        run the timeline server"
   echo "  version                               print the version"
   echo " or"
@@ -162,6 +163,10 @@ case "${COMMAND}" in
     CLASS='org.apache.hadoop.yarn.server.sharedcachemanager.SharedCacheManager'
     YARN_OPTS="$YARN_OPTS $YARN_SHAREDCACHEMANAGER_OPTS"
   ;;
+  scmadmin)
+    CLASS='org.apache.hadoop.yarn.client.SCMAdmin'
+    YARN_OPTS="$YARN_OPTS $YARN_CLIENT_OPTS"
+  ;;
   version)
     CLASS=org.apache.hadoop.util.VersionInfo
     hadoop_debug "Append YARN_CLIENT_OPTS onto YARN_OPTS"

http://git-wip-us.apache.org/repos/asf/hadoop/blob/78968155/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/pom.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/pom.xml
index 5e2278d..a763d39 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/pom.xml
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/pom.xml
@@ -97,6 +97,7 @@
                   <include>application_history_client.proto</include>
                   <include>server/application_history_server.proto</include>
                   <include>client_SCM_protocol.proto</include>
+                  <include>server/SCM_Admin_protocol.proto</include>
                 </includes>
               </source>
               <output>${project.build.directory}/generated-sources/java</output>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/78968155/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 41f85ef..f0f88d8 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
@@ -1401,6 +1401,18 @@ public class YarnConfiguration extends Configuration {
   public static final String DEFAULT_SCM_APP_CHECKER_CLASS =
       "org.apache.hadoop.yarn.server.sharedcachemanager.RemoteAppChecker";
 
+  /** The address of the SCM admin interface. */
+  public static final String SCM_ADMIN_ADDRESS =
+      SHARED_CACHE_PREFIX + "admin.address";
+  public static final int DEFAULT_SCM_ADMIN_PORT = 8047;
+  public static final String DEFAULT_SCM_ADMIN_ADDRESS =
+      "0.0.0.0:" + DEFAULT_SCM_ADMIN_PORT;
+
+  /** Number of threads used to handle SCM admin interface. */
+  public static final String SCM_ADMIN_CLIENT_THREAD_COUNT =
+      SHARED_CACHE_PREFIX + "admin.thread-count";
+  public static final int DEFAULT_SCM_ADMIN_CLIENT_THREAD_COUNT = 1;
+
   // In-memory SCM store configuration
   
   public static final String IN_MEMORY_STORE_PREFIX =

http://git-wip-us.apache.org/repos/asf/hadoop/blob/78968155/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/server/api/SCMAdminProtocol.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/server/api/SCMAdminProtocol.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/server/api/SCMAdminProtocol.java
new file mode 100644
index 0000000..5c791fa
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/server/api/SCMAdminProtocol.java
@@ -0,0 +1,53 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.server.api;
+
+import java.io.IOException;
+
+import org.apache.hadoop.classification.InterfaceAudience.Public;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.yarn.server.api.protocolrecords.RunSharedCacheCleanerTaskRequest;
+import org.apache.hadoop.yarn.server.api.protocolrecords.RunSharedCacheCleanerTaskResponse;
+import org.apache.hadoop.yarn.exceptions.YarnException;
+
+/**
+ * <p>
+ * The protocol between administrators and the <code>SharedCacheManager</code>
+ * </p>
+ */
+@Public
+@Unstable
+public interface SCMAdminProtocol {
+  /**
+   * <p>
+   * The method used by administrators to ask SCM to run cleaner task right away
+   * </p>
+   *
+   * @param request request <code>SharedCacheManager</code> to run a cleaner task
+   * @return <code>SharedCacheManager</code> returns an empty response
+   *         on success and throws an exception on rejecting the request
+   * @throws YarnException
+   * @throws IOException
+   */
+  @Public
+  @Unstable
+  public RunSharedCacheCleanerTaskResponse runCleanerTask(
+      RunSharedCacheCleanerTaskRequest request) throws YarnException, IOException;
+  
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/78968155/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/server/api/SCMAdminProtocolPB.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/server/api/SCMAdminProtocolPB.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/server/api/SCMAdminProtocolPB.java
new file mode 100644
index 0000000..93a2c67
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/server/api/SCMAdminProtocolPB.java
@@ -0,0 +1,31 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.yarn.server.api;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.ipc.ProtocolInfo;
+import org.apache.hadoop.yarn.proto.SCMAdminProtocol.SCMAdminProtocolService;
+
+@Private
+@Unstable
+@ProtocolInfo(protocolName = "org.apache.hadoop.yarn.server.api.SCMAdminProtocolPB",
+    protocolVersion = 1)
+public interface SCMAdminProtocolPB extends
+    SCMAdminProtocolService.BlockingInterface {
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/78968155/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/RunSharedCacheCleanerTaskRequest.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/RunSharedCacheCleanerTaskRequest.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/RunSharedCacheCleanerTaskRequest.java
new file mode 100644
index 0000000..5b4b110
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/RunSharedCacheCleanerTaskRequest.java
@@ -0,0 +1,37 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.server.api.protocolrecords;
+
+import org.apache.hadoop.classification.InterfaceAudience.Public;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+
+/**
+ * <p>
+ * The request from admin to ask the <code>SharedCacheManager</code> to run
+ * cleaner service right away.
+ * </p>
+ * 
+ * <p>
+ * Currently, this is empty.
+ * </p>
+ */
+@Public
+@Unstable
+public abstract class RunSharedCacheCleanerTaskRequest {
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/78968155/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/RunSharedCacheCleanerTaskResponse.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/RunSharedCacheCleanerTaskResponse.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/RunSharedCacheCleanerTaskResponse.java
new file mode 100644
index 0000000..1d32d32
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/RunSharedCacheCleanerTaskResponse.java
@@ -0,0 +1,58 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.server.api.protocolrecords;
+
+import org.apache.hadoop.classification.InterfaceAudience.Public;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+
+/**
+ * <p>
+ * The response to admin from the <code>SharedCacheManager</code> when
+ * is asked to run the cleaner service.
+ * </p>
+ * 
+ * <p>
+ * Currently, this is empty.
+ * </p>
+ */
+@Public
+@Unstable
+public abstract class RunSharedCacheCleanerTaskResponse {
+
+  /**
+   * Get whether or not the shared cache manager has accepted the request.
+   * Shared cache manager will reject the request if there is an ongoing task
+   * 
+   * @return boolean True if the request has been accepted, false otherwise.
+   */
+  @Public
+  @Unstable
+  public abstract boolean getAccepted();
+
+  /**
+   * Set whether or not the shared cache manager has accepted the request Shared
+   * cache manager will reject the request if there is an ongoing task
+   * 
+   * @param b True if the request has been accepted, false otherwise.
+   */
+  @Public
+  @Unstable
+  public abstract void setAccepted(boolean b);
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/78968155/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/server/SCM_Admin_protocol.proto
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/server/SCM_Admin_protocol.proto b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/server/SCM_Admin_protocol.proto
new file mode 100644
index 0000000..4e46c57
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/server/SCM_Admin_protocol.proto
@@ -0,0 +1,29 @@
+/**
+ * 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.
+ */
+
+option java_package = "org.apache.hadoop.yarn.proto";
+option java_outer_classname = "SCMAdminProtocol";
+option java_generic_services = true;
+option java_generate_equals_and_hash = true;
+package hadoop.yarn;
+
+import "yarn_service_protos.proto";
+
+service SCMAdminProtocolService {
+  rpc runCleanerTask (RunSharedCacheCleanerTaskRequestProto) returns (RunSharedCacheCleanerTaskResponseProto);
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/78968155/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_service_protos.proto
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_service_protos.proto b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_service_protos.proto
index 10f5b9f..94e73e1 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_service_protos.proto
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_service_protos.proto
@@ -354,3 +354,14 @@ message ReservationDeleteRequestProto {
 
 message ReservationDeleteResponseProto {
 }
+
+//////////////////////////////////////////////////////
+/////// SCM_Admin_Protocol //////////////////////////
+//////////////////////////////////////////////////////
+
+message RunSharedCacheCleanerTaskRequestProto {
+}
+
+message RunSharedCacheCleanerTaskResponseProto {
+  optional bool accepted = 1;
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/78968155/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/SCMAdmin.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/SCMAdmin.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/SCMAdmin.java
new file mode 100644
index 0000000..1e45c5a
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/SCMAdmin.java
@@ -0,0 +1,183 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.client;
+
+import java.io.IOException;
+import java.net.InetSocketAddress;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.conf.Configured;
+import org.apache.hadoop.ipc.RemoteException;
+import org.apache.hadoop.util.Tool;
+import org.apache.hadoop.util.ToolRunner;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.hadoop.yarn.factories.RecordFactory;
+import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
+import org.apache.hadoop.yarn.ipc.YarnRPC;
+import org.apache.hadoop.yarn.server.api.SCMAdminProtocol;
+import org.apache.hadoop.yarn.server.api.protocolrecords.RunSharedCacheCleanerTaskRequest;
+import org.apache.hadoop.yarn.server.api.protocolrecords.RunSharedCacheCleanerTaskResponse;
+
+public class SCMAdmin extends Configured implements Tool {
+
+  private final RecordFactory recordFactory = 
+    RecordFactoryProvider.getRecordFactory(null);
+
+  public SCMAdmin() {
+    super();
+  }
+
+  public SCMAdmin(Configuration conf) {
+    super(conf);
+  }
+
+  private static void printHelp(String cmd) {
+    String summary = "scmadmin is the command to execute shared cache manager" +
+        "administrative commands.\n" +
+        "The full syntax is: \n\n" +
+        "hadoop scmadmin" +
+        " [-runCleanerTask]" +
+        " [-help [cmd]]\n";
+
+    String runCleanerTask =
+        "-runCleanerTask: Run cleaner task right away.\n";
+
+    String help = "-help [cmd]: \tDisplays help for the given command or all commands if none\n" +
+        "\t\tis specified.\n";
+
+    if ("runCleanerTask".equals(cmd)) {
+      System.out.println(runCleanerTask);
+    } else if ("help".equals(cmd)) {
+      System.out.println(help);
+    } else {
+      System.out.println(summary);
+      System.out.println(runCleanerTask);
+      System.out.println(help);
+      System.out.println();
+      ToolRunner.printGenericCommandUsage(System.out);
+    }
+  }
+
+  /**
+   * Displays format of commands.
+   * @param cmd The command that is being executed.
+   */
+  private static void printUsage(String cmd) {
+    if ("-runCleanerTask".equals(cmd)) {
+      System.err.println("Usage: yarn scmadmin" + " [-runCleanerTask]");
+    } else {
+      System.err.println("Usage: yarn scmadmin");
+      System.err.println("           [-runCleanerTask]");
+      System.err.println("           [-help [cmd]]");
+      System.err.println();
+      ToolRunner.printGenericCommandUsage(System.err);
+    }
+  }
+
+  protected SCMAdminProtocol createSCMAdminProtocol() throws IOException {
+    // Get the current configuration
+    final YarnConfiguration conf = new YarnConfiguration(getConf());
+
+    // Create the admin client
+    final InetSocketAddress addr = conf.getSocketAddr(
+        YarnConfiguration.SCM_ADMIN_ADDRESS,
+        YarnConfiguration.DEFAULT_SCM_ADMIN_ADDRESS,
+        YarnConfiguration.DEFAULT_SCM_ADMIN_PORT);
+    final YarnRPC rpc = YarnRPC.create(conf);
+    SCMAdminProtocol scmAdminProtocol =
+        (SCMAdminProtocol) rpc.getProxy(SCMAdminProtocol.class, addr, conf);
+    return scmAdminProtocol;
+  }
+  
+  private int runCleanerTask() throws YarnException, IOException {
+    // run cleaner task right away
+    SCMAdminProtocol scmAdminProtocol = createSCMAdminProtocol();
+    RunSharedCacheCleanerTaskRequest request =
+      recordFactory.newRecordInstance(RunSharedCacheCleanerTaskRequest.class);
+    RunSharedCacheCleanerTaskResponse response =
+        scmAdminProtocol.runCleanerTask(request);
+    if (response.getAccepted()) {
+      System.out.println("request accepted by shared cache manager");
+      return 0;
+    } else {
+      System.out.println("request rejected by shared cache manager");
+      return 1;
+    }
+  }
+
+  @Override
+  public int run(String[] args) throws Exception {
+    if (args.length < 1) {
+      printUsage("");
+      return -1;
+    }
+
+    int i = 0;
+    String cmd = args[i++];
+
+    try {
+      if ("-runCleanerTask".equals(cmd)) {
+        if (args.length != 1) {
+          printUsage(cmd);
+          return -1;
+        } else {
+          return runCleanerTask();
+        }
+      } else if ("-help".equals(cmd)) {
+        if (i < args.length) {
+          printUsage(args[i]);
+        } else {
+          printHelp("");
+        }
+        return 0;
+      } else {
+        System.err.println(cmd.substring(1) + ": Unknown command");
+        printUsage("");
+        return -1;
+      }
+
+    } catch (IllegalArgumentException arge) {
+      System.err.println(cmd.substring(1) + ": " + arge.getLocalizedMessage());
+      printUsage(cmd);
+    } catch (RemoteException e) {
+      //
+      // This is a error returned by hadoop server. Print
+      // out the first line of the error message, ignore the stack trace.
+      try {
+        String[] content;
+        content = e.getLocalizedMessage().split("\n");
+        System.err.println(cmd.substring(1) + ": "
+                           + content[0]);
+      } catch (Exception ex) {
+        System.err.println(cmd.substring(1) + ": "
+                           + ex.getLocalizedMessage());
+      }
+    } catch (Exception e) {
+      System.err.println(cmd.substring(1) + ": "
+                         + e.getLocalizedMessage());
+    }
+    return -1;
+  }
+
+  public static void main(String[] args) throws Exception {
+    int result = ToolRunner.run(new SCMAdmin(), args);
+    System.exit(result);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/78968155/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/server/api/impl/pb/client/SCMAdminProtocolPBClientImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/server/api/impl/pb/client/SCMAdminProtocolPBClientImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/server/api/impl/pb/client/SCMAdminProtocolPBClientImpl.java
new file mode 100644
index 0000000..4dd06d6
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/server/api/impl/pb/client/SCMAdminProtocolPBClientImpl.java
@@ -0,0 +1,73 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.server.api.impl.pb.client;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.net.InetSocketAddress;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.ipc.ProtobufRpcEngine;
+import org.apache.hadoop.ipc.RPC;
+import org.apache.hadoop.yarn.server.api.SCMAdminProtocol;
+import org.apache.hadoop.yarn.server.api.SCMAdminProtocolPB;
+import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.RunSharedCacheCleanerTaskRequestPBImpl;
+import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.RunSharedCacheCleanerTaskResponsePBImpl;
+import org.apache.hadoop.yarn.server.api.protocolrecords.RunSharedCacheCleanerTaskRequest;
+import org.apache.hadoop.yarn.server.api.protocolrecords.RunSharedCacheCleanerTaskResponse;
+import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.hadoop.yarn.ipc.RPCUtil;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos;
+
+import com.google.protobuf.ServiceException;
+
+public class SCMAdminProtocolPBClientImpl implements SCMAdminProtocol,
+    Closeable {
+
+  private SCMAdminProtocolPB proxy;
+
+  public SCMAdminProtocolPBClientImpl(long clientVersion,
+      InetSocketAddress addr, Configuration conf) throws IOException {
+    RPC.setProtocolEngine(conf, SCMAdminProtocolPB.class,
+        ProtobufRpcEngine.class);
+    proxy = RPC.getProxy(SCMAdminProtocolPB.class, clientVersion, addr, conf);
+  }
+
+  @Override
+  public void close() {
+    if (this.proxy != null) {
+      RPC.stopProxy(this.proxy);
+    }
+  }
+
+  @Override
+  public RunSharedCacheCleanerTaskResponse runCleanerTask(
+      RunSharedCacheCleanerTaskRequest request) throws YarnException,
+      IOException {
+    YarnServiceProtos.RunSharedCacheCleanerTaskRequestProto requestProto =
+        ((RunSharedCacheCleanerTaskRequestPBImpl) request).getProto();
+    try {
+      return new RunSharedCacheCleanerTaskResponsePBImpl(proxy.runCleanerTask(null,
+          requestProto));
+    } catch (ServiceException e) {
+      RPCUtil.unwrapAndThrowException(e);
+      return null;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/78968155/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/server/api/impl/pb/service/SCMAdminProtocolPBServiceImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/server/api/impl/pb/service/SCMAdminProtocolPBServiceImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/server/api/impl/pb/service/SCMAdminProtocolPBServiceImpl.java
new file mode 100644
index 0000000..7d8c578
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/server/api/impl/pb/service/SCMAdminProtocolPBServiceImpl.java
@@ -0,0 +1,57 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.server.api.impl.pb.service;
+
+import java.io.IOException;
+
+import org.apache.hadoop.yarn.server.api.SCMAdminProtocol;
+import org.apache.hadoop.yarn.server.api.SCMAdminProtocolPB;
+import org.apache.hadoop.yarn.server.api.protocolrecords.RunSharedCacheCleanerTaskResponse;
+import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.RunSharedCacheCleanerTaskRequestPBImpl;
+import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.RunSharedCacheCleanerTaskResponsePBImpl;
+import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos.RunSharedCacheCleanerTaskRequestProto;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos.RunSharedCacheCleanerTaskResponseProto;
+
+import com.google.protobuf.RpcController;
+import com.google.protobuf.ServiceException;
+
+public class SCMAdminProtocolPBServiceImpl implements SCMAdminProtocolPB {
+
+  private SCMAdminProtocol real;
+  
+  public SCMAdminProtocolPBServiceImpl(SCMAdminProtocol impl) {
+    this.real = impl;
+  }
+
+  @Override
+  public RunSharedCacheCleanerTaskResponseProto runCleanerTask(RpcController controller,
+      RunSharedCacheCleanerTaskRequestProto proto) throws ServiceException {
+    RunSharedCacheCleanerTaskRequestPBImpl request =
+        new RunSharedCacheCleanerTaskRequestPBImpl(proto);
+    try {
+      RunSharedCacheCleanerTaskResponse response = real.runCleanerTask(request);
+      return ((RunSharedCacheCleanerTaskResponsePBImpl) response).getProto();
+    } catch (YarnException e) {
+      throw new ServiceException(e);
+    } catch (IOException e) {
+      throw new ServiceException(e);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/78968155/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/RunSharedCacheCleanerTaskRequestPBImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/RunSharedCacheCleanerTaskRequestPBImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/RunSharedCacheCleanerTaskRequestPBImpl.java
new file mode 100644
index 0000000..91d1b5c
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/RunSharedCacheCleanerTaskRequestPBImpl.java
@@ -0,0 +1,53 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb;
+
+import org.apache.hadoop.yarn.server.api.protocolrecords.RunSharedCacheCleanerTaskRequest;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos.RunSharedCacheCleanerTaskRequestProto;
+
+public class RunSharedCacheCleanerTaskRequestPBImpl extends
+    RunSharedCacheCleanerTaskRequest {
+  RunSharedCacheCleanerTaskRequestProto proto =
+      RunSharedCacheCleanerTaskRequestProto.getDefaultInstance();
+  RunSharedCacheCleanerTaskRequestProto.Builder builder = null;
+  boolean viaProto = false;
+
+  public RunSharedCacheCleanerTaskRequestPBImpl() {
+    builder = RunSharedCacheCleanerTaskRequestProto.newBuilder();
+  }
+
+  public RunSharedCacheCleanerTaskRequestPBImpl(
+      RunSharedCacheCleanerTaskRequestProto proto) {
+    this.proto = proto;
+    viaProto = true;
+  }
+
+  public RunSharedCacheCleanerTaskRequestProto getProto() {
+    proto = viaProto ? proto : builder.build();
+    viaProto = true;
+    return proto;
+  }
+
+  private void maybeInitBuilder() {
+    if (viaProto || builder == null) {
+      builder = RunSharedCacheCleanerTaskRequestProto.newBuilder(proto);
+    }
+    viaProto = false;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/78968155/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/RunSharedCacheCleanerTaskResponsePBImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/RunSharedCacheCleanerTaskResponsePBImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/RunSharedCacheCleanerTaskResponsePBImpl.java
new file mode 100644
index 0000000..02c4f31
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/RunSharedCacheCleanerTaskResponsePBImpl.java
@@ -0,0 +1,66 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb;
+
+import org.apache.hadoop.yarn.server.api.protocolrecords.RunSharedCacheCleanerTaskResponse;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos.RunSharedCacheCleanerTaskResponseProto;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos.RunSharedCacheCleanerTaskResponseProtoOrBuilder;
+
+public class RunSharedCacheCleanerTaskResponsePBImpl extends
+    RunSharedCacheCleanerTaskResponse {
+  RunSharedCacheCleanerTaskResponseProto proto =
+      RunSharedCacheCleanerTaskResponseProto.getDefaultInstance();
+  RunSharedCacheCleanerTaskResponseProto.Builder builder = null;
+  boolean viaProto = false;
+
+  public RunSharedCacheCleanerTaskResponsePBImpl() {
+    builder = RunSharedCacheCleanerTaskResponseProto.newBuilder();
+  }
+
+  public RunSharedCacheCleanerTaskResponsePBImpl(
+      RunSharedCacheCleanerTaskResponseProto proto) {
+    this.proto = proto;
+    viaProto = true;
+  }
+
+  @Override
+  public boolean getAccepted() {
+    RunSharedCacheCleanerTaskResponseProtoOrBuilder p = viaProto ? proto : builder;
+    return (p.hasAccepted()) ? p.getAccepted() : false;
+  }
+
+  @Override
+  public void setAccepted(boolean b) {
+    maybeInitBuilder();
+    builder.setAccepted(b);
+  }
+
+  public RunSharedCacheCleanerTaskResponseProto getProto() {
+    proto = viaProto ? proto : builder.build();
+    viaProto = true;
+    return proto;
+  }
+
+  private void maybeInitBuilder() {
+    if (viaProto || builder == null) {
+      builder = RunSharedCacheCleanerTaskResponseProto.newBuilder(proto);
+    }
+    viaProto = false;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/78968155/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml
index 54c4dbc..73a6b5d 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml
@@ -1431,6 +1431,18 @@
     <name>yarn.sharedcache.store.in-memory.check-period-mins</name>
     <value>720</value>
   </property>
+  
+  <property>
+    <description>The address of the admin interface in the SCM (shared cache manager)</description>
+    <name>yarn.sharedcache.admin.address</name>
+    <value>0.0.0.0:8047</value>
+  </property>
+
+  <property>
+    <description>The number of threads used to handle SCM admin interface (1 by default)</description>
+    <name>yarn.sharedcache.admin.thread-count</name>
+    <value>1</value>
+  </property>
 
   <property>
     <description>The frequency at which a cleaner task runs.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/78968155/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-sharedcachemanager/src/main/java/org/apache/hadoop/yarn/server/sharedcachemanager/SCMAdminProtocolService.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-sharedcachemanager/src/main/java/org/apache/hadoop/yarn/server/sharedcachemanager/SCMAdminProtocolService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-sharedcachemanager/src/main/java/org/apache/hadoop/yarn/server/sharedcachemanager/SCMAdminProtocolService.java
new file mode 100644
index 0000000..3ecca02
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-sharedcachemanager/src/main/java/org/apache/hadoop/yarn/server/sharedcachemanager/SCMAdminProtocolService.java
@@ -0,0 +1,146 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.server.sharedcachemanager;
+
+import java.io.IOException;
+import java.net.InetSocketAddress;
+
+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;
+import org.apache.hadoop.ipc.Server;
+import org.apache.hadoop.security.AccessControlException;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.security.authorize.AccessControlList;
+import org.apache.hadoop.service.AbstractService;
+import org.apache.hadoop.yarn.server.api.SCMAdminProtocol;
+import org.apache.hadoop.yarn.server.api.protocolrecords.RunSharedCacheCleanerTaskRequest;
+import org.apache.hadoop.yarn.server.api.protocolrecords.RunSharedCacheCleanerTaskResponse;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.hadoop.yarn.factories.RecordFactory;
+import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
+import org.apache.hadoop.yarn.ipc.RPCUtil;
+import org.apache.hadoop.yarn.ipc.YarnRPC;
+
+/**
+ * This service handles all SCMAdminProtocol rpc calls from administrators
+ * to the shared cache manager.
+ */
+@Private
+@Unstable
+public class SCMAdminProtocolService extends AbstractService implements
+    SCMAdminProtocol {
+
+  private static final Log LOG = LogFactory.getLog(SCMAdminProtocolService.class);
+
+  private final RecordFactory recordFactory = RecordFactoryProvider
+      .getRecordFactory(null);
+
+  private Server server;
+  InetSocketAddress clientBindAddress;
+  private final CleanerService cleanerService;
+  private AccessControlList adminAcl;
+
+  public SCMAdminProtocolService(CleanerService cleanerService) {
+    super(SCMAdminProtocolService.class.getName());
+    this.cleanerService = cleanerService;
+  }
+
+  @Override
+  protected void serviceInit(Configuration conf) throws Exception {
+    this.clientBindAddress = getBindAddress(conf);
+    adminAcl = new AccessControlList(conf.get(
+        YarnConfiguration.YARN_ADMIN_ACL,
+        YarnConfiguration.DEFAULT_YARN_ADMIN_ACL));
+    super.serviceInit(conf);
+  }
+
+  InetSocketAddress getBindAddress(Configuration conf) {
+    return conf.getSocketAddr(YarnConfiguration.SCM_ADMIN_ADDRESS,
+        YarnConfiguration.DEFAULT_SCM_ADMIN_ADDRESS,
+        YarnConfiguration.DEFAULT_SCM_ADMIN_PORT);
+  }
+
+  @Override
+  protected void serviceStart() throws Exception {
+    Configuration conf = getConfig();
+    YarnRPC rpc = YarnRPC.create(conf);
+    this.server =
+        rpc.getServer(SCMAdminProtocol.class, this,
+            clientBindAddress,
+            conf, null, // Secret manager null for now (security not supported)
+            conf.getInt(YarnConfiguration.SCM_ADMIN_CLIENT_THREAD_COUNT,
+                YarnConfiguration.DEFAULT_SCM_ADMIN_CLIENT_THREAD_COUNT));
+
+    // TODO: Enable service authorization (see YARN-2774)
+
+    this.server.start();
+    clientBindAddress =
+        conf.updateConnectAddr(YarnConfiguration.SCM_ADMIN_ADDRESS,
+            server.getListenerAddress());
+
+    super.serviceStart();
+  }
+
+  @Override
+  protected void serviceStop() throws Exception {
+    if (this.server != null) {
+      this.server.stop();
+    }
+
+    super.serviceStop();
+  }
+
+  private void checkAcls(String method) throws YarnException {
+    UserGroupInformation user;
+    try {
+      user = UserGroupInformation.getCurrentUser();
+    } catch (IOException ioe) {
+      LOG.warn("Couldn't get current user", ioe);
+      throw RPCUtil.getRemoteException(ioe);
+    }
+
+    if (!adminAcl.isUserAllowed(user)) {
+      LOG.warn("User " + user.getShortUserName() + " doesn't have permission" +
+          " to call '" + method + "'");
+
+      throw RPCUtil.getRemoteException(
+          new AccessControlException("User " + user.getShortUserName() +
+          " doesn't have permission" + " to call '" + method + "'"));
+    }
+    LOG.info("SCM Admin: " + method + " invoked by user " +
+        user.getShortUserName());
+  }
+
+  @Override
+  public RunSharedCacheCleanerTaskResponse runCleanerTask(
+      RunSharedCacheCleanerTaskRequest request) throws YarnException {
+    checkAcls("runCleanerTask");
+    RunSharedCacheCleanerTaskResponse response =
+        recordFactory.newRecordInstance(RunSharedCacheCleanerTaskResponse.class);
+    this.cleanerService.runCleanerTask();
+    // if we are here, then we have submitted the request to the cleaner
+    // service, ack the request to the admin client
+    response.setAccepted(true);
+    return response;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/78968155/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-sharedcachemanager/src/main/java/org/apache/hadoop/yarn/server/sharedcachemanager/SharedCacheManager.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-sharedcachemanager/src/main/java/org/apache/hadoop/yarn/server/sharedcachemanager/SharedCacheManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-sharedcachemanager/src/main/java/org/apache/hadoop/yarn/server/sharedcachemanager/SharedCacheManager.java
index c54e470..5c33b2b 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-sharedcachemanager/src/main/java/org/apache/hadoop/yarn/server/sharedcachemanager/SharedCacheManager.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-sharedcachemanager/src/main/java/org/apache/hadoop/yarn/server/sharedcachemanager/SharedCacheManager.java
@@ -74,6 +74,9 @@ public class SharedCacheManager extends CompositeService {
     ClientProtocolService cps = createClientProtocolService(store);
     addService(cps);
 
+    SCMAdminProtocolService saps = createSCMAdminProtocolService(cs);
+    addService(saps);
+
     // init metrics
     DefaultMetricsSystem.initialize("SharedCacheManager");
     JvmMetrics.initSingleton("SharedCacheManager", null);
@@ -113,6 +116,11 @@ public class SharedCacheManager extends CompositeService {
     return new ClientProtocolService(store);
   }
 
+  private SCMAdminProtocolService createSCMAdminProtocolService(
+      CleanerService cleanerService) {
+    return new SCMAdminProtocolService(cleanerService);
+  }
+
   @Override
   protected void serviceStop() throws Exception {
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/78968155/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-sharedcachemanager/src/test/java/org/apache/hadoop/yarn/server/sharedcachemanager/TestSCMAdminProtocolService.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-sharedcachemanager/src/test/java/org/apache/hadoop/yarn/server/sharedcachemanager/TestSCMAdminProtocolService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-sharedcachemanager/src/test/java/org/apache/hadoop/yarn/server/sharedcachemanager/TestSCMAdminProtocolService.java
new file mode 100644
index 0000000..e6cf15f
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-sharedcachemanager/src/test/java/org/apache/hadoop/yarn/server/sharedcachemanager/TestSCMAdminProtocolService.java
@@ -0,0 +1,135 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.server.sharedcachemanager;
+
+import static org.junit.Assert.assertEquals;
+import static org.mockito.Matchers.any;
+import static org.mockito.Matchers.isA;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.spy;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.doNothing;
+import static org.mockito.Mockito.when;
+
+import java.io.IOException;
+import java.net.InetSocketAddress;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.ipc.RPC;
+import org.apache.hadoop.yarn.server.api.SCMAdminProtocol;
+import org.apache.hadoop.yarn.server.api.protocolrecords.RunSharedCacheCleanerTaskRequest;
+import org.apache.hadoop.yarn.server.api.protocolrecords.RunSharedCacheCleanerTaskResponse;
+import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.RunSharedCacheCleanerTaskResponsePBImpl;
+import org.apache.hadoop.yarn.client.SCMAdmin;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.factories.RecordFactory;
+import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
+import org.apache.hadoop.yarn.ipc.YarnRPC;
+import org.apache.hadoop.yarn.server.sharedcachemanager.store.InMemorySCMStore;
+import org.apache.hadoop.yarn.server.sharedcachemanager.store.SCMStore;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+/**
+ * Basic unit tests for the SCM Admin Protocol Service and SCMAdmin.
+ */
+public class TestSCMAdminProtocolService {
+
+  static SCMAdminProtocolService service;
+  static SCMAdminProtocol SCMAdminProxy;
+  static SCMAdminProtocol mockAdmin;
+  static SCMAdmin adminCLI;
+  static SCMStore store;
+  static CleanerService cleaner;
+  private final RecordFactory recordFactory = RecordFactoryProvider
+      .getRecordFactory(null);
+
+  @Before
+  public void startUp() {
+    Configuration conf = new Configuration();
+    conf.set(YarnConfiguration.SCM_STORE_CLASS,
+        InMemorySCMStore.class.getName());
+
+    cleaner = mock(CleanerService.class);
+
+    service = spy(new SCMAdminProtocolService(cleaner));
+    service.init(conf);
+    service.start();
+
+    YarnRPC rpc = YarnRPC.create(new Configuration());
+
+    InetSocketAddress scmAddress =
+        conf.getSocketAddr(YarnConfiguration.SCM_ADMIN_ADDRESS,
+            YarnConfiguration.DEFAULT_SCM_ADMIN_ADDRESS,
+            YarnConfiguration.DEFAULT_SCM_ADMIN_PORT);
+
+    SCMAdminProxy =
+        (SCMAdminProtocol) rpc.getProxy(SCMAdminProtocol.class, scmAddress,
+            conf);
+
+    mockAdmin = mock(SCMAdminProtocol.class);
+    adminCLI = new SCMAdmin(new Configuration()) {
+      @Override
+      protected SCMAdminProtocol createSCMAdminProtocol() throws IOException {
+        return mockAdmin;
+      }
+    };
+  }
+
+  @After
+  public void cleanUpTest() {
+    if (service != null) {
+      service.stop();
+    }
+
+    if (SCMAdminProxy != null) {
+      RPC.stopProxy(SCMAdminProxy);
+    }
+  }
+
+  @Test
+  public void testRunCleanerTask() throws Exception {
+    doNothing().when(cleaner).runCleanerTask();
+    RunSharedCacheCleanerTaskRequest request =
+        recordFactory.newRecordInstance(RunSharedCacheCleanerTaskRequest.class);
+    RunSharedCacheCleanerTaskResponse response = SCMAdminProxy.runCleanerTask(request);
+    Assert.assertTrue("cleaner task request isn't accepted", response.getAccepted());
+    verify(service, times(1)).runCleanerTask(any(RunSharedCacheCleanerTaskRequest.class));
+  }
+
+  @Test
+  public void testRunCleanerTaskCLI() throws Exception {
+    String[] args = { "-runCleanerTask" };
+    RunSharedCacheCleanerTaskResponse rp =
+        new RunSharedCacheCleanerTaskResponsePBImpl();
+    rp.setAccepted(true);
+    when(mockAdmin.runCleanerTask(isA(RunSharedCacheCleanerTaskRequest.class)))
+        .thenReturn(rp);
+    assertEquals(0, adminCLI.run(args));
+    rp.setAccepted(false);
+    when(mockAdmin.runCleanerTask(isA(RunSharedCacheCleanerTaskRequest.class)))
+        .thenReturn(rp);
+    assertEquals(1, adminCLI.run(args));
+    verify(mockAdmin, times(2)).runCleanerTask(
+        any(RunSharedCacheCleanerTaskRequest.class));
+  }
+}


[12/29] hadoop git commit: YARN-2301. Improved yarn container command. Contributed by Naganarasimha G R

Posted by vi...@apache.org.
YARN-2301. Improved yarn container command. Contributed by Naganarasimha G R


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

Branch: refs/heads/HDFS-EC
Commit: 258623ff8bb1a1057ae3501d4f20982d5a59ea34
Parents: 565b0e6
Author: Jian He <ji...@apache.org>
Authored: Thu Dec 4 12:51:15 2014 -0800
Committer: Jian He <ji...@apache.org>
Committed: Thu Dec 4 12:53:18 2014 -0800

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                 |  2 +
 .../hadoop/yarn/client/cli/ApplicationCLI.java  |  8 +++-
 .../hadoop/yarn/client/cli/TestYarnCLI.java     | 41 +++++++++++++-------
 .../yarn/server/resourcemanager/RMContext.java  |  3 ++
 .../server/resourcemanager/RMContextImpl.java   | 11 ++++++
 .../server/resourcemanager/ResourceManager.java |  2 +
 .../rmcontainer/RMContainerImpl.java            |  9 ++++-
 .../resourcemanager/TestClientRMService.java    |  1 +
 .../rmcontainer/TestRMContainerImpl.java        |  6 ++-
 9 files changed, 63 insertions(+), 20 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/258623ff/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index 30b9260..f032b4f 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -121,6 +121,8 @@ Release 2.7.0 - UNRELEASED
     YARN-1156. Enhance NodeManager AllocatedGB and AvailableGB metrics 
     for aggregation of decimal values. (Tsuyoshi OZAWA via junping_du)
 
+    YARN-2301. Improved yarn container command. (Naganarasimha G R via jianhe)
+
   OPTIMIZATIONS
 
   BUG FIXES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/258623ff/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/ApplicationCLI.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/ApplicationCLI.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/ApplicationCLI.java
index a847cd5..83d212d 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/ApplicationCLI.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/ApplicationCLI.java
@@ -44,6 +44,7 @@ import org.apache.hadoop.yarn.api.records.YarnApplicationState;
 import org.apache.hadoop.yarn.exceptions.ApplicationNotFoundException;
 import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.util.ConverterUtils;
+import org.apache.hadoop.yarn.util.Times;
 
 import com.google.common.annotations.VisibleForTesting;
 
@@ -536,8 +537,11 @@ public class ApplicationCLI extends YarnCLI {
     writer.printf(CONTAINER_PATTERN, "Container-Id", "Start Time",
         "Finish Time", "State", "Host", "LOG-URL");
     for (ContainerReport containerReport : appsReport) {
-      writer.printf(CONTAINER_PATTERN, containerReport.getContainerId(),
-          containerReport.getCreationTime(), containerReport.getFinishTime(),
+      writer.printf(
+          CONTAINER_PATTERN,
+          containerReport.getContainerId(),
+          Times.format(containerReport.getCreationTime()),
+          Times.format(containerReport.getFinishTime()),      
           containerReport.getContainerState(), containerReport
               .getAssignedNode(), containerReport.getLogUrl());
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/258623ff/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestYarnCLI.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestYarnCLI.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestYarnCLI.java
index 9d9a86a..194d7d1 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestYarnCLI.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestYarnCLI.java
@@ -32,19 +32,17 @@ import java.io.ByteArrayOutputStream;
 import java.io.IOException;
 import java.io.PrintStream;
 import java.io.PrintWriter;
+import java.text.DateFormat;
+import java.text.SimpleDateFormat;
 import java.util.ArrayList;
 import java.util.Date;
 import java.util.EnumSet;
-import java.util.HashMap;
 import java.util.HashSet;
 import java.util.List;
-import java.util.Map;
 import java.util.Set;
 
 import org.apache.commons.cli.Options;
 import org.apache.commons.lang.time.DateFormatUtils;
-import org.apache.hadoop.yarn.api.protocolrecords.GetClusterNodeLabelsResponse;
-import org.apache.hadoop.yarn.api.protocolrecords.GetNodesToLabelsResponse;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptReport;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
@@ -65,9 +63,7 @@ import org.apache.hadoop.yarn.api.records.YarnApplicationAttemptState;
 import org.apache.hadoop.yarn.api.records.YarnApplicationState;
 import org.apache.hadoop.yarn.client.api.YarnClient;
 import org.apache.hadoop.yarn.exceptions.ApplicationNotFoundException;
-import org.apache.hadoop.yarn.nodelabels.CommonNodeLabelsManager;
 import org.apache.hadoop.yarn.util.Records;
-import org.jboss.netty.logging.CommonsLoggerFactory;
 import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
@@ -257,25 +253,31 @@ public class TestYarnCLI {
         applicationId, 1);
     ContainerId containerId = ContainerId.newContainerId(attemptId, 1);
     ContainerId containerId1 = ContainerId.newContainerId(attemptId, 2);
+    ContainerId containerId2 = ContainerId.newContainerId(attemptId, 3);
+    long time1=1234,time2=5678;
     ContainerReport container = ContainerReport.newInstance(containerId, null,
-        NodeId.newInstance("host", 1234), Priority.UNDEFINED, 1234, 5678,
+        NodeId.newInstance("host", 1234), Priority.UNDEFINED, time1, time2,
         "diagnosticInfo", "logURL", 0, ContainerState.COMPLETE);
     ContainerReport container1 = ContainerReport.newInstance(containerId1, null,
-        NodeId.newInstance("host", 1234), Priority.UNDEFINED, 1234, 5678,
+        NodeId.newInstance("host", 1234), Priority.UNDEFINED, time1, time2,
         "diagnosticInfo", "logURL", 0, ContainerState.COMPLETE);
+    ContainerReport container2 = ContainerReport.newInstance(containerId2, null,
+        NodeId.newInstance("host", 1234), Priority.UNDEFINED, time1,0,
+        "diagnosticInfo", "", 0, ContainerState.RUNNING);
     List<ContainerReport> reports = new ArrayList<ContainerReport>();
     reports.add(container);
     reports.add(container1);
+    reports.add(container2);
+    DateFormat dateFormat=new SimpleDateFormat("EEE MMM dd HH:mm:ss Z yyyy");
     when(client.getContainers(any(ApplicationAttemptId.class))).thenReturn(
         reports);
     int result = cli.run(new String[] { "container", "-list",
         attemptId.toString() });
     assertEquals(0, result);
     verify(client).getContainers(attemptId);
-    Log.info(sysOutStream.toString());
     ByteArrayOutputStream baos = new ByteArrayOutputStream();
     PrintWriter pw = new PrintWriter(baos);
-    pw.println("Total number of containers :2");
+    pw.println("Total number of containers :3");
     pw.print("                  Container-Id");
     pw.print("\t          Start Time");
     pw.print("\t         Finish Time");
@@ -283,19 +285,30 @@ public class TestYarnCLI {
     pw.print("\t                Host");
     pw.println("\t                            LOG-URL");
     pw.print(" container_1234_0005_01_000001");
-    pw.print("\t                1234");
-    pw.print("\t                5678");
+    pw.print("\t"+dateFormat.format(new Date(time1)));
+    pw.print("\t"+dateFormat.format(new Date(time2)));
     pw.print("\t            COMPLETE");
     pw.print("\t           host:1234");
     pw.println("\t                             logURL");
     pw.print(" container_1234_0005_01_000002");
-    pw.print("\t                1234");
-    pw.print("\t                5678");
+    pw.print("\t"+dateFormat.format(new Date(time1)));
+    pw.print("\t"+dateFormat.format(new Date(time2)));
     pw.print("\t            COMPLETE");
     pw.print("\t           host:1234");
     pw.println("\t                             logURL");
+    pw.print(" container_1234_0005_01_000003");
+    pw.print("\t"+dateFormat.format(new Date(time1)));
+    pw.print("\t                 N/A");
+    pw.print("\t             RUNNING");
+    pw.print("\t           host:1234");
+    pw.println("\t                                   ");
     pw.close();
     String appReportStr = baos.toString("UTF-8");
+    Log.info("ExpectedOutput");
+    Log.info("["+appReportStr+"]");
+    Log.info("OutputFrom command");
+    String actualOutput = sysOutStream.toString();
+    Log.info("["+actualOutput+"]");
     Assert.assertEquals(appReportStr, sysOutStream.toString());
   }
   

http://git-wip-us.apache.org/repos/asf/hadoop/blob/258623ff/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMContext.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMContext.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMContext.java
index 56984e6..ecf6166 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMContext.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMContext.java
@@ -21,6 +21,7 @@ package org.apache.hadoop.yarn.server.resourcemanager;
 import java.nio.ByteBuffer;
 import java.util.concurrent.ConcurrentMap;
 
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.ha.HAServiceProtocol.HAServiceState;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.NodeId;
@@ -121,4 +122,6 @@ public interface RMContext {
   ReservationSystem getReservationSystem();
 
   boolean isSchedulerReadyForAllocatingContainers();
+  
+  Configuration getYarnConfiguration();
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/258623ff/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMContextImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMContextImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMContextImpl.java
index 55d7667..ebf2fe4 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMContextImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMContextImpl.java
@@ -22,6 +22,7 @@ import java.nio.ByteBuffer;
 import java.util.concurrent.ConcurrentMap;
 
 import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.ha.HAServiceProtocol;
 import org.apache.hadoop.ha.HAServiceProtocol.HAServiceState;
@@ -65,6 +66,8 @@ public class RMContextImpl implements RMContext {
 
   private RMActiveServiceContext activeServiceContext;
 
+  private Configuration yarnConfiguration;
+
   /**
    * Default constructor. To be used in conjunction with setter methods for
    * individual fields.
@@ -399,4 +402,12 @@ public class RMContextImpl implements RMContext {
     this.activeServiceContext = activeServiceContext;
   }
 
+  @Override
+  public Configuration getYarnConfiguration() {
+    return this.yarnConfiguration;
+  }
+
+  public void setYarnConfiguration(Configuration yarnConfiguration) {
+    this.yarnConfiguration=yarnConfiguration;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/258623ff/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceManager.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/ResourceManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceManager.java
index 9662c39..775d08a 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceManager.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceManager.java
@@ -246,6 +246,8 @@ public class ResourceManager extends CompositeService implements Recoverable {
     addService(adminService);
     rmContext.setRMAdminService(adminService);
     
+    rmContext.setYarnConfiguration(conf);
+    
     createAndInitActiveServices();
 
     webAppAddress = WebAppUtils.getWebAppBindURL(this.conf,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/258623ff/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainerImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainerImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainerImpl.java
index 479734a..e37d8fd 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainerImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainerImpl.java
@@ -285,8 +285,13 @@ public class RMContainerImpl implements RMContainer {
   public String getLogURL() {
     try {
       readLock.lock();
-      return WebAppUtils.getRunningLogURL("//" + container.getNodeHttpAddress(),
-          ConverterUtils.toString(containerId), user);
+      StringBuilder logURL = new StringBuilder();
+      logURL.append(WebAppUtils.getHttpSchemePrefix(rmContext
+          .getYarnConfiguration()));
+      logURL.append(WebAppUtils.getRunningLogURL(
+          container.getNodeHttpAddress(), ConverterUtils.toString(containerId),
+          user));
+      return logURL.toString();
     } finally {
       readLock.unlock();
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/258623ff/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestClientRMService.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/TestClientRMService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestClientRMService.java
index 15aca42..a344e9a 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestClientRMService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestClientRMService.java
@@ -443,6 +443,7 @@ public class TestClientRMService {
     ConcurrentHashMap<ApplicationId, RMApp> apps = getRMApps(rmContext,
         yarnScheduler);
     when(rmContext.getRMApps()).thenReturn(apps);
+    when(rmContext.getYarnConfiguration()).thenReturn(new Configuration());
     RMAppManager appManager = new RMAppManager(rmContext, yarnScheduler, null,
         mock(ApplicationACLsManager.class), new Configuration());
     when(rmContext.getDispatcher().getEventHandler()).thenReturn(

http://git-wip-us.apache.org/repos/asf/hadoop/blob/258623ff/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/TestRMContainerImpl.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/rmcontainer/TestRMContainerImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/TestRMContainerImpl.java
index 76cdcae..21aba3b 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/TestRMContainerImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/TestRMContainerImpl.java
@@ -107,6 +107,7 @@ public class TestRMContainerImpl {
     when(rmContext.getRMApplicationHistoryWriter()).thenReturn(writer);
     when(rmContext.getRMApps()).thenReturn(rmApps);
     when(rmContext.getSystemMetricsPublisher()).thenReturn(publisher);
+    when(rmContext.getYarnConfiguration()).thenReturn(new YarnConfiguration());
     RMContainer rmContainer = new RMContainerImpl(container, appAttemptId,
         nodeId, "user", rmContext);
 
@@ -130,7 +131,7 @@ public class TestRMContainerImpl {
         RMContainerEventType.LAUNCHED));
     drainDispatcher.await();
     assertEquals(RMContainerState.RUNNING, rmContainer.getState());
-    assertEquals("//host:3465/node/containerlogs/container_1_0001_01_000001/user",
+    assertEquals("http://host:3465/node/containerlogs/container_1_0001_01_000001/user",
         rmContainer.getLogURL());
 
     // In RUNNING state. Verify RELEASED and associated actions.
@@ -196,6 +197,7 @@ public class TestRMContainerImpl {
     when(rmContext.getContainerAllocationExpirer()).thenReturn(expirer);
     when(rmContext.getRMApplicationHistoryWriter()).thenReturn(writer);
     when(rmContext.getSystemMetricsPublisher()).thenReturn(publisher);
+    when(rmContext.getYarnConfiguration()).thenReturn(new YarnConfiguration());
     RMContainer rmContainer = new RMContainerImpl(container, appAttemptId,
         nodeId, "user", rmContext);
 
@@ -220,7 +222,7 @@ public class TestRMContainerImpl {
         RMContainerEventType.LAUNCHED));
     drainDispatcher.await();
     assertEquals(RMContainerState.RUNNING, rmContainer.getState());
-    assertEquals("//host:3465/node/containerlogs/container_1_0001_01_000001/user",
+    assertEquals("http://host:3465/node/containerlogs/container_1_0001_01_000001/user",
         rmContainer.getLogURL());
 
     // In RUNNING state. Verify EXPIRE and associated actions.


[26/29] hadoop git commit: HDFS-7476. Consolidate ACL-related operations to a single class. Contributed by Haohui Mai.

Posted by vi...@apache.org.
HDFS-7476. Consolidate ACL-related operations to a single class. Contributed by Haohui Mai.


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

Branch: refs/heads/HDFS-EC
Commit: 9297f980c2de8886ff970946a2513e6890cd5552
Parents: e227fb8
Author: cnauroth <cn...@apache.org>
Authored: Sat Dec 6 14:20:00 2014 -0800
Committer: cnauroth <cn...@apache.org>
Committed: Sat Dec 6 14:20:00 2014 -0800

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |   3 +
 .../hadoop/hdfs/server/namenode/AclStorage.java |  33 ---
 .../hadoop/hdfs/server/namenode/FSDirAclOp.java | 244 +++++++++++++++++++
 .../hdfs/server/namenode/FSDirectory.java       | 158 ++----------
 .../hdfs/server/namenode/FSEditLogLoader.java   |   2 +-
 .../hdfs/server/namenode/FSNamesystem.java      | 119 ++-------
 .../hdfs/server/namenode/TestAuditLogger.java   |  79 ++----
 7 files changed, 318 insertions(+), 320 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/9297f980/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index 87b02c4..769be43 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -438,6 +438,9 @@ Release 2.7.0 - UNRELEASED
     HDFS-7459. Consolidate cache-related implementation in FSNamesystem into
     a single class. (wheat9)
 
+    HDFS-7476. Consolidate ACL-related operations to a single class.
+    (wheat9 via cnauroth)
+
   OPTIMIZATIONS
 
     HDFS-7454. Reduce memory footprint for AclEntries in NameNode.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9297f980/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/AclStorage.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/AclStorage.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/AclStorage.java
index ac30597..a866046 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/AclStorage.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/AclStorage.java
@@ -241,39 +241,6 @@ final class AclStorage {
   }
 
   /**
-   * Completely removes the ACL from an inode.
-   *
-   * @param inode INode to update
-   * @param snapshotId int latest snapshot ID of inode
-   * @throws QuotaExceededException if quota limit is exceeded
-   */
-  public static void removeINodeAcl(INode inode, int snapshotId)
-      throws QuotaExceededException {
-    AclFeature f = inode.getAclFeature();
-    if (f == null) {
-      return;
-    }
-
-    FsPermission perm = inode.getFsPermission();
-    List<AclEntry> featureEntries = getEntriesFromAclFeature(f);
-    if (featureEntries.get(0).getScope() == AclEntryScope.ACCESS) {
-      // Restore group permissions from the feature's entry to permission
-      // bits, overwriting the mask, which is not part of a minimal ACL.
-      AclEntry groupEntryKey = new AclEntry.Builder()
-          .setScope(AclEntryScope.ACCESS).setType(AclEntryType.GROUP).build();
-      int groupEntryIndex = Collections.binarySearch(featureEntries,
-          groupEntryKey, AclTransformation.ACL_ENTRY_COMPARATOR);
-      assert groupEntryIndex >= 0;
-      FsAction groupPerm = featureEntries.get(groupEntryIndex).getPermission();
-      FsPermission newPerm = new FsPermission(perm.getUserAction(), groupPerm,
-          perm.getOtherAction(), perm.getStickyBit());
-      inode.setPermission(newPerm, snapshotId);
-    }
-
-    inode.removeAclFeature(snapshotId);
-  }
-
-  /**
    * Updates an inode with a new ACL.  This method takes a full logical ACL and
    * stores the entries to the inode's {@link FsPermission} and
    * {@link AclFeature}.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9297f980/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirAclOp.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirAclOp.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirAclOp.java
new file mode 100644
index 0000000..ac899aa
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirAclOp.java
@@ -0,0 +1,244 @@
+/**
+ * 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.fs.permission.AclEntry;
+import org.apache.hadoop.fs.permission.AclEntryScope;
+import org.apache.hadoop.fs.permission.AclEntryType;
+import org.apache.hadoop.fs.permission.AclStatus;
+import org.apache.hadoop.fs.permission.FsAction;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.protocol.AclException;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.List;
+
+class FSDirAclOp {
+  static HdfsFileStatus modifyAclEntries(
+      FSDirectory fsd, final String srcArg, List<AclEntry> aclSpec)
+      throws IOException {
+    String src = srcArg;
+    checkAclsConfigFlag(fsd);
+    FSPermissionChecker pc = fsd.getPermissionChecker();
+    byte[][] pathComponents = FSDirectory.getPathComponentsForReservedPath(src);
+    src = fsd.resolvePath(pc, src, pathComponents);
+    fsd.writeLock();
+    try {
+      INodesInPath iip = fsd.getINodesInPath4Write(
+          FSDirectory.normalizePath(src), true);
+      fsd.checkOwner(pc, iip);
+      INode inode = FSDirectory.resolveLastINode(src, iip);
+      int snapshotId = iip.getLatestSnapshotId();
+      List<AclEntry> existingAcl = AclStorage.readINodeLogicalAcl(inode);
+      List<AclEntry> newAcl = AclTransformation.mergeAclEntries(
+          existingAcl, aclSpec);
+      AclStorage.updateINodeAcl(inode, newAcl, snapshotId);
+      fsd.getEditLog().logSetAcl(src, newAcl);
+    } finally {
+      fsd.writeUnlock();
+    }
+    return fsd.getAuditFileInfo(src, false);
+  }
+
+  static HdfsFileStatus removeAclEntries(
+      FSDirectory fsd, final String srcArg, List<AclEntry> aclSpec)
+      throws IOException {
+    String src = srcArg;
+    checkAclsConfigFlag(fsd);
+    FSPermissionChecker pc = fsd.getPermissionChecker();
+    byte[][] pathComponents = FSDirectory.getPathComponentsForReservedPath(src);
+    src = fsd.resolvePath(pc, src, pathComponents);
+    fsd.writeLock();
+    try {
+      INodesInPath iip = fsd.getINodesInPath4Write(
+          FSDirectory.normalizePath(src), true);
+      fsd.checkOwner(pc, iip);
+      INode inode = FSDirectory.resolveLastINode(src, iip);
+      int snapshotId = iip.getLatestSnapshotId();
+      List<AclEntry> existingAcl = AclStorage.readINodeLogicalAcl(inode);
+      List<AclEntry> newAcl = AclTransformation.filterAclEntriesByAclSpec(
+        existingAcl, aclSpec);
+      AclStorage.updateINodeAcl(inode, newAcl, snapshotId);
+      fsd.getEditLog().logSetAcl(src, newAcl);
+    } finally {
+      fsd.writeUnlock();
+    }
+    return fsd.getAuditFileInfo(src, false);
+  }
+
+  static HdfsFileStatus removeDefaultAcl(FSDirectory fsd, final String srcArg)
+      throws IOException {
+    String src = srcArg;
+    checkAclsConfigFlag(fsd);
+    FSPermissionChecker pc = fsd.getPermissionChecker();
+    byte[][] pathComponents = FSDirectory.getPathComponentsForReservedPath(src);
+    src = fsd.resolvePath(pc, src, pathComponents);
+    fsd.writeLock();
+    try {
+      INodesInPath iip = fsd.getINodesInPath4Write(
+          FSDirectory.normalizePath(src), true);
+      fsd.checkOwner(pc, iip);
+      INode inode = FSDirectory.resolveLastINode(src, iip);
+      int snapshotId = iip.getLatestSnapshotId();
+      List<AclEntry> existingAcl = AclStorage.readINodeLogicalAcl(inode);
+      List<AclEntry> newAcl = AclTransformation.filterDefaultAclEntries(
+        existingAcl);
+      AclStorage.updateINodeAcl(inode, newAcl, snapshotId);
+      fsd.getEditLog().logSetAcl(src, newAcl);
+    } finally {
+      fsd.writeUnlock();
+    }
+    return fsd.getAuditFileInfo(src, false);
+  }
+
+  static HdfsFileStatus removeAcl(FSDirectory fsd, final String srcArg)
+      throws IOException {
+    String src = srcArg;
+    checkAclsConfigFlag(fsd);
+    FSPermissionChecker pc = fsd.getPermissionChecker();
+    byte[][] pathComponents = FSDirectory.getPathComponentsForReservedPath(src);
+    src = fsd.resolvePath(pc, src, pathComponents);
+    fsd.writeLock();
+    try {
+      INodesInPath iip = fsd.getINodesInPath4Write(src);
+      fsd.checkOwner(pc, iip);
+      unprotectedRemoveAcl(fsd, src);
+    } finally {
+      fsd.writeUnlock();
+    }
+    fsd.getEditLog().logSetAcl(src, AclFeature.EMPTY_ENTRY_LIST);
+    return fsd.getAuditFileInfo(src, false);
+  }
+
+  static HdfsFileStatus setAcl(
+      FSDirectory fsd, final String srcArg, List<AclEntry> aclSpec)
+      throws IOException {
+    String src = srcArg;
+    checkAclsConfigFlag(fsd);
+    byte[][] pathComponents = FSDirectory.getPathComponentsForReservedPath(src);
+    FSPermissionChecker pc = fsd.getPermissionChecker();
+    src = fsd.resolvePath(pc, src, pathComponents);
+    fsd.writeLock();
+    try {
+      INodesInPath iip = fsd.getINodesInPath4Write(src);
+      fsd.checkOwner(pc, iip);
+      List<AclEntry> newAcl = unprotectedSetAcl(fsd, src, aclSpec);
+      fsd.getEditLog().logSetAcl(src, newAcl);
+    } finally {
+      fsd.writeUnlock();
+    }
+    return fsd.getAuditFileInfo(src, false);
+  }
+
+  static AclStatus getAclStatus(
+      FSDirectory fsd, String src) throws IOException {
+    checkAclsConfigFlag(fsd);
+    FSPermissionChecker pc = fsd.getPermissionChecker();
+    byte[][] pathComponents = FSDirectory.getPathComponentsForReservedPath(src);
+    src = fsd.resolvePath(pc, src, pathComponents);
+    String srcs = FSDirectory.normalizePath(src);
+    fsd.readLock();
+    try {
+      // There is no real inode for the path ending in ".snapshot", so return a
+      // non-null, unpopulated AclStatus.  This is similar to getFileInfo.
+      if (srcs.endsWith(HdfsConstants.SEPARATOR_DOT_SNAPSHOT_DIR) &&
+          fsd.getINode4DotSnapshot(srcs) != null) {
+        return new AclStatus.Builder().owner("").group("").build();
+      }
+      INodesInPath iip = fsd.getINodesInPath(srcs, true);
+      if (fsd.isPermissionEnabled()) {
+        fsd.checkTraverse(pc, iip);
+      }
+      INode inode = FSDirectory.resolveLastINode(srcs, iip);
+      int snapshotId = iip.getPathSnapshotId();
+      List<AclEntry> acl = AclStorage.readINodeAcl(inode, snapshotId);
+      return new AclStatus.Builder()
+          .owner(inode.getUserName()).group(inode.getGroupName())
+          .stickyBit(inode.getFsPermission(snapshotId).getStickyBit())
+          .addEntries(acl).build();
+    } finally {
+      fsd.readUnlock();
+    }
+  }
+
+  static List<AclEntry> unprotectedSetAcl(
+      FSDirectory fsd, String src, List<AclEntry> aclSpec)
+      throws IOException {
+    // ACL removal is logged to edits as OP_SET_ACL with an empty list.
+    if (aclSpec.isEmpty()) {
+      unprotectedRemoveAcl(fsd, src);
+      return AclFeature.EMPTY_ENTRY_LIST;
+    }
+
+    assert fsd.hasWriteLock();
+    INodesInPath iip = fsd.getINodesInPath4Write(FSDirectory.normalizePath
+        (src), true);
+    INode inode = FSDirectory.resolveLastINode(src, iip);
+    int snapshotId = iip.getLatestSnapshotId();
+    List<AclEntry> existingAcl = AclStorage.readINodeLogicalAcl(inode);
+    List<AclEntry> newAcl = AclTransformation.replaceAclEntries(existingAcl,
+      aclSpec);
+    AclStorage.updateINodeAcl(inode, newAcl, snapshotId);
+    return newAcl;
+  }
+
+  private static void checkAclsConfigFlag(FSDirectory fsd) throws AclException {
+    if (!fsd.isAclsEnabled()) {
+      throw new AclException(String.format(
+          "The ACL operation has been rejected.  "
+              + "Support for ACLs has been disabled by setting %s to false.",
+          DFSConfigKeys.DFS_NAMENODE_ACLS_ENABLED_KEY));
+    }
+  }
+
+  private static void unprotectedRemoveAcl(FSDirectory fsd, String src)
+      throws IOException {
+    assert fsd.hasWriteLock();
+    INodesInPath iip = fsd.getINodesInPath4Write(
+        FSDirectory.normalizePath(src), true);
+    INode inode = FSDirectory.resolveLastINode(src, iip);
+    int snapshotId = iip.getLatestSnapshotId();
+    AclFeature f = inode.getAclFeature();
+    if (f == null) {
+      return;
+    }
+
+    FsPermission perm = inode.getFsPermission();
+    List<AclEntry> featureEntries = AclStorage.getEntriesFromAclFeature(f);
+    if (featureEntries.get(0).getScope() == AclEntryScope.ACCESS) {
+      // Restore group permissions from the feature's entry to permission
+      // bits, overwriting the mask, which is not part of a minimal ACL.
+      AclEntry groupEntryKey = new AclEntry.Builder()
+          .setScope(AclEntryScope.ACCESS).setType(AclEntryType.GROUP).build();
+      int groupEntryIndex = Collections.binarySearch(
+          featureEntries, groupEntryKey,
+          AclTransformation.ACL_ENTRY_COMPARATOR);
+      assert groupEntryIndex >= 0;
+      FsAction groupPerm = featureEntries.get(groupEntryIndex).getPermission();
+      FsPermission newPerm = new FsPermission(perm.getUserAction(), groupPerm,
+          perm.getOtherAction(), perm.getStickyBit());
+      inode.setPermission(newPerm, snapshotId);
+    }
+
+    inode.removeAclFeature(snapshotId);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9297f980/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 444589e..82741ce 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
@@ -48,7 +48,6 @@ import org.apache.hadoop.fs.UnresolvedLinkException;
 import org.apache.hadoop.fs.XAttr;
 import org.apache.hadoop.fs.XAttrSetFlag;
 import org.apache.hadoop.fs.permission.AclEntry;
-import org.apache.hadoop.fs.permission.AclStatus;
 import org.apache.hadoop.fs.permission.FsAction;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.permission.PermissionStatus;
@@ -143,6 +142,12 @@ public class FSDirectory implements Closeable {
   private final ReentrantReadWriteLock dirLock;
 
   private final boolean isPermissionEnabled;
+  /**
+   * Support for ACLs is controlled by a configuration flag. If the
+   * configuration flag is false, then the NameNode will reject all
+   * ACL-related operations.
+   */
+  private final boolean aclsEnabled;
   private final String fsOwnerShortUserName;
   private final String supergroup;
   private final INodeId inodeId;
@@ -204,7 +209,10 @@ public class FSDirectory implements Closeable {
     this.supergroup = conf.get(
       DFSConfigKeys.DFS_PERMISSIONS_SUPERUSERGROUP_KEY,
       DFSConfigKeys.DFS_PERMISSIONS_SUPERUSERGROUP_DEFAULT);
-
+    this.aclsEnabled = conf.getBoolean(
+        DFSConfigKeys.DFS_NAMENODE_ACLS_ENABLED_KEY,
+        DFSConfigKeys.DFS_NAMENODE_ACLS_ENABLED_DEFAULT);
+    LOG.info("ACLs enabled? " + aclsEnabled);
     int configuredLimit = conf.getInt(
         DFSConfigKeys.DFS_LIST_LIMIT, DFSConfigKeys.DFS_LIST_LIMIT_DEFAULT);
     this.lsLimit = configuredLimit>0 ?
@@ -263,6 +271,9 @@ public class FSDirectory implements Closeable {
   boolean isPermissionEnabled() {
     return isPermissionEnabled;
   }
+  boolean isAclsEnabled() {
+    return aclsEnabled;
+  }
 
   int getLsLimit() {
     return lsLimit;
@@ -1549,140 +1560,6 @@ public class FSDirectory implements Closeable {
     return addINode(path, symlink) ? symlink : null;
   }
 
-  List<AclEntry> modifyAclEntries(String src, List<AclEntry> aclSpec) throws IOException {
-    writeLock();
-    try {
-      return unprotectedModifyAclEntries(src, aclSpec);
-    } finally {
-      writeUnlock();
-    }
-  }
-
-  private List<AclEntry> unprotectedModifyAclEntries(String src,
-      List<AclEntry> aclSpec) throws IOException {
-    assert hasWriteLock();
-    INodesInPath iip = getINodesInPath4Write(normalizePath(src), true);
-    INode inode = resolveLastINode(src, iip);
-    int snapshotId = iip.getLatestSnapshotId();
-    List<AclEntry> existingAcl = AclStorage.readINodeLogicalAcl(inode);
-    List<AclEntry> newAcl = AclTransformation.mergeAclEntries(existingAcl,
-      aclSpec);
-    AclStorage.updateINodeAcl(inode, newAcl, snapshotId);
-    return newAcl;
-  }
-
-  List<AclEntry> removeAclEntries(String src, List<AclEntry> aclSpec) throws IOException {
-    writeLock();
-    try {
-      return unprotectedRemoveAclEntries(src, aclSpec);
-    } finally {
-      writeUnlock();
-    }
-  }
-
-  private List<AclEntry> unprotectedRemoveAclEntries(String src,
-      List<AclEntry> aclSpec) throws IOException {
-    assert hasWriteLock();
-    INodesInPath iip = getINodesInPath4Write(normalizePath(src), true);
-    INode inode = resolveLastINode(src, iip);
-    int snapshotId = iip.getLatestSnapshotId();
-    List<AclEntry> existingAcl = AclStorage.readINodeLogicalAcl(inode);
-    List<AclEntry> newAcl = AclTransformation.filterAclEntriesByAclSpec(
-      existingAcl, aclSpec);
-    AclStorage.updateINodeAcl(inode, newAcl, snapshotId);
-    return newAcl;
-  }
-
-  List<AclEntry> removeDefaultAcl(String src) throws IOException {
-    writeLock();
-    try {
-      return unprotectedRemoveDefaultAcl(src);
-    } finally {
-      writeUnlock();
-    }
-  }
-
-  private List<AclEntry> unprotectedRemoveDefaultAcl(String src)
-      throws IOException {
-    assert hasWriteLock();
-    INodesInPath iip = getINodesInPath4Write(normalizePath(src), true);
-    INode inode = resolveLastINode(src, iip);
-    int snapshotId = iip.getLatestSnapshotId();
-    List<AclEntry> existingAcl = AclStorage.readINodeLogicalAcl(inode);
-    List<AclEntry> newAcl = AclTransformation.filterDefaultAclEntries(
-      existingAcl);
-    AclStorage.updateINodeAcl(inode, newAcl, snapshotId);
-    return newAcl;
-  }
-
-  void removeAcl(String src) throws IOException {
-    writeLock();
-    try {
-      unprotectedRemoveAcl(src);
-    } finally {
-      writeUnlock();
-    }
-  }
-
-  private void unprotectedRemoveAcl(String src) throws IOException {
-    assert hasWriteLock();
-    INodesInPath iip = getINodesInPath4Write(normalizePath(src), true);
-    INode inode = resolveLastINode(src, iip);
-    int snapshotId = iip.getLatestSnapshotId();
-    AclStorage.removeINodeAcl(inode, snapshotId);
-  }
-
-  List<AclEntry> setAcl(String src, List<AclEntry> aclSpec) throws IOException {
-    writeLock();
-    try {
-      return unprotectedSetAcl(src, aclSpec);
-    } finally {
-      writeUnlock();
-    }
-  }
-
-  List<AclEntry> unprotectedSetAcl(String src, List<AclEntry> aclSpec)
-      throws IOException {
-    // ACL removal is logged to edits as OP_SET_ACL with an empty list.
-    if (aclSpec.isEmpty()) {
-      unprotectedRemoveAcl(src);
-      return AclFeature.EMPTY_ENTRY_LIST;
-    }
-
-    assert hasWriteLock();
-    INodesInPath iip = getINodesInPath4Write(normalizePath(src), true);
-    INode inode = resolveLastINode(src, iip);
-    int snapshotId = iip.getLatestSnapshotId();
-    List<AclEntry> existingAcl = AclStorage.readINodeLogicalAcl(inode);
-    List<AclEntry> newAcl = AclTransformation.replaceAclEntries(existingAcl,
-      aclSpec);
-    AclStorage.updateINodeAcl(inode, newAcl, snapshotId);
-    return newAcl;
-  }
-
-  AclStatus getAclStatus(String src) throws IOException {
-    String srcs = normalizePath(src);
-    readLock();
-    try {
-      // There is no real inode for the path ending in ".snapshot", so return a
-      // non-null, unpopulated AclStatus.  This is similar to getFileInfo.
-      if (srcs.endsWith(HdfsConstants.SEPARATOR_DOT_SNAPSHOT_DIR) &&
-          getINode4DotSnapshot(srcs) != null) {
-        return new AclStatus.Builder().owner("").group("").build();
-      }
-      INodesInPath iip = getLastINodeInPath(srcs, true);
-      INode inode = resolveLastINode(src, iip);
-      int snapshotId = iip.getPathSnapshotId();
-      List<AclEntry> acl = AclStorage.readINodeAcl(inode, snapshotId);
-      return new AclStatus.Builder()
-          .owner(inode.getUserName()).group(inode.getGroupName())
-          .stickyBit(inode.getFsPermission(snapshotId).getStickyBit())
-          .addEntries(acl).build();
-    } finally {
-      readUnlock();
-    }
-  }
-
   /**
    * Removes a list of XAttrs from an inode at a path.
    *
@@ -2065,9 +1942,10 @@ public class FSDirectory implements Closeable {
     return null;
   }
 
-  private static INode resolveLastINode(String src, INodesInPath iip)
+  static INode resolveLastINode(String src, INodesInPath iip)
       throws FileNotFoundException {
-    INode inode = iip.getLastINode();
+    INode[] inodes = iip.getINodes();
+    INode inode = inodes[inodes.length - 1];
     if (inode == null)
       throw new FileNotFoundException("cannot find " + src);
     return inode;
@@ -2246,8 +2124,8 @@ public class FSDirectory implements Closeable {
   }
 
   /** @return the {@link INodesInPath} containing only the last inode. */
-  private INodesInPath getLastINodeInPath(String path, boolean resolveLink
-  ) throws UnresolvedLinkException {
+  INodesInPath getLastINodeInPath(
+      String path, boolean resolveLink) throws UnresolvedLinkException {
     return INodesInPath.resolve(rootDir, INode.getPathComponents(path), 1,
             resolveLink);
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9297f980/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java
index f60f142..d63545b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java
@@ -823,7 +823,7 @@ public class FSEditLogLoader {
     }
     case OP_SET_ACL: {
       SetAclOp setAclOp = (SetAclOp) op;
-      fsDir.unprotectedSetAcl(setAclOp.src, setAclOp.aclEntries);
+      FSDirAclOp.unprotectedSetAcl(fsDir, setAclOp.src, setAclOp.aclEntries);
       break;
     }
     case OP_SET_XATTR: {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9297f980/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 982798f..e9ce78c 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
@@ -532,7 +532,6 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
 
   private final RetryCache retryCache;
 
-  private final boolean aclsEnabled;
   private final boolean xattrsEnabled;
   private final int xattrMaxSize;
 
@@ -851,10 +850,6 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
         auditLoggers.get(0) instanceof DefaultAuditLogger;
       this.retryCache = ignoreRetryCache ? null : initRetryCache(conf);
 
-      this.aclsEnabled = conf.getBoolean(
-          DFSConfigKeys.DFS_NAMENODE_ACLS_ENABLED_KEY,
-          DFSConfigKeys.DFS_NAMENODE_ACLS_ENABLED_DEFAULT);
-      LOG.info("ACLs enabled? " + aclsEnabled);
       this.xattrsEnabled = conf.getBoolean(
           DFSConfigKeys.DFS_NAMENODE_XATTRS_ENABLED_KEY,
           DFSConfigKeys.DFS_NAMENODE_XATTRS_ENABLED_DEFAULT);
@@ -7731,158 +7726,105 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     return results;
   }
 
-  void modifyAclEntries(final String srcArg, List<AclEntry> aclSpec)
+  void modifyAclEntries(final String src, List<AclEntry> aclSpec)
       throws IOException {
-    String src = srcArg;
-    checkAclsConfigFlag();
-    HdfsFileStatus resultingStat = null;
-    FSPermissionChecker pc = getPermissionChecker();
+    HdfsFileStatus auditStat = null;
     checkOperation(OperationCategory.WRITE);
-    byte[][] pathComponents = FSDirectory.getPathComponentsForReservedPath(src);
     writeLock();
     try {
       checkOperation(OperationCategory.WRITE);
       checkNameNodeSafeMode("Cannot modify ACL entries on " + src);
-      src = dir.resolvePath(pc, src, pathComponents);
-      final INodesInPath iip = dir.getINodesInPath4Write(src);
-      dir.checkOwner(pc, iip);
-      List<AclEntry> newAcl = dir.modifyAclEntries(src, aclSpec);
-      getEditLog().logSetAcl(src, newAcl);
-      resultingStat = getAuditFileInfo(src, false);
+      auditStat = FSDirAclOp.modifyAclEntries(dir, src, aclSpec);
     } catch (AccessControlException e) {
-      logAuditEvent(false, "modifyAclEntries", srcArg);
+      logAuditEvent(false, "modifyAclEntries", src);
       throw e;
     } finally {
       writeUnlock();
     }
     getEditLog().logSync();
-    logAuditEvent(true, "modifyAclEntries", srcArg, null, resultingStat);
+    logAuditEvent(true, "modifyAclEntries", src, null, auditStat);
   }
 
-  void removeAclEntries(final String srcArg, List<AclEntry> aclSpec)
+  void removeAclEntries(final String src, List<AclEntry> aclSpec)
       throws IOException {
-    String src = srcArg;
-    checkAclsConfigFlag();
-    HdfsFileStatus resultingStat = null;
-    FSPermissionChecker pc = getPermissionChecker();
     checkOperation(OperationCategory.WRITE);
-    byte[][] pathComponents = FSDirectory.getPathComponentsForReservedPath(src);
+    HdfsFileStatus auditStat = null;
     writeLock();
     try {
       checkOperation(OperationCategory.WRITE);
       checkNameNodeSafeMode("Cannot remove ACL entries on " + src);
-      src = dir.resolvePath(pc, src, pathComponents);
-      final INodesInPath iip = dir.getINodesInPath4Write(src);
-      dir.checkOwner(pc, iip);
-      List<AclEntry> newAcl = dir.removeAclEntries(src, aclSpec);
-      getEditLog().logSetAcl(src, newAcl);
-      resultingStat = getAuditFileInfo(src, false);
+      auditStat = FSDirAclOp.removeAclEntries(dir, src, aclSpec);
     } catch (AccessControlException e) {
-      logAuditEvent(false, "removeAclEntries", srcArg);
+      logAuditEvent(false, "removeAclEntries", src);
       throw e;
     } finally {
       writeUnlock();
     }
     getEditLog().logSync();
-    logAuditEvent(true, "removeAclEntries", srcArg, null, resultingStat);
+    logAuditEvent(true, "removeAclEntries", src, null, auditStat);
   }
 
-  void removeDefaultAcl(final String srcArg) throws IOException {
-    String src = srcArg;
-    checkAclsConfigFlag();
-    HdfsFileStatus resultingStat = null;
-    FSPermissionChecker pc = getPermissionChecker();
+  void removeDefaultAcl(final String src) throws IOException {
+    HdfsFileStatus auditStat = null;
     checkOperation(OperationCategory.WRITE);
-    byte[][] pathComponents = FSDirectory.getPathComponentsForReservedPath(src);
     writeLock();
     try {
       checkOperation(OperationCategory.WRITE);
       checkNameNodeSafeMode("Cannot remove default ACL entries on " + src);
-      src = dir.resolvePath(pc, src, pathComponents);
-      final INodesInPath iip = dir.getINodesInPath4Write(src);
-      dir.checkOwner(pc, iip);
-      List<AclEntry> newAcl = dir.removeDefaultAcl(src);
-      getEditLog().logSetAcl(src, newAcl);
-      resultingStat = getAuditFileInfo(src, false);
+      auditStat = FSDirAclOp.removeDefaultAcl(dir, src);
     } catch (AccessControlException e) {
-      logAuditEvent(false, "removeDefaultAcl", srcArg);
+      logAuditEvent(false, "removeDefaultAcl", src);
       throw e;
     } finally {
       writeUnlock();
     }
     getEditLog().logSync();
-    logAuditEvent(true, "removeDefaultAcl", srcArg, null, resultingStat);
+    logAuditEvent(true, "removeDefaultAcl", src, null, auditStat);
   }
 
-  void removeAcl(final String srcArg) throws IOException {
-    String src = srcArg;
-    checkAclsConfigFlag();
-    HdfsFileStatus resultingStat = null;
-    FSPermissionChecker pc = getPermissionChecker();
+  void removeAcl(final String src) throws IOException {
+    HdfsFileStatus auditStat = null;
     checkOperation(OperationCategory.WRITE);
-    byte[][] pathComponents = FSDirectory.getPathComponentsForReservedPath(src);
     writeLock();
     try {
       checkOperation(OperationCategory.WRITE);
       checkNameNodeSafeMode("Cannot remove ACL on " + src);
-      src = dir.resolvePath(pc, src, pathComponents);
-      final INodesInPath iip = dir.getINodesInPath4Write(src);
-      dir.checkOwner(pc, iip);
-      dir.removeAcl(src);
-      getEditLog().logSetAcl(src, AclFeature.EMPTY_ENTRY_LIST);
-      resultingStat = getAuditFileInfo(src, false);
+      auditStat = FSDirAclOp.removeAcl(dir, src);
     } catch (AccessControlException e) {
-      logAuditEvent(false, "removeAcl", srcArg);
+      logAuditEvent(false, "removeAcl", src);
       throw e;
     } finally {
       writeUnlock();
     }
     getEditLog().logSync();
-    logAuditEvent(true, "removeAcl", srcArg, null, resultingStat);
+    logAuditEvent(true, "removeAcl", src, null, auditStat);
   }
 
-  void setAcl(final String srcArg, List<AclEntry> aclSpec) throws IOException {
-    String src = srcArg;
-    checkAclsConfigFlag();
-    HdfsFileStatus resultingStat = null;
-    FSPermissionChecker pc = getPermissionChecker();
+  void setAcl(final String src, List<AclEntry> aclSpec) throws IOException {
+    HdfsFileStatus auditStat = null;
     checkOperation(OperationCategory.WRITE);
-    byte[][] pathComponents = FSDirectory.getPathComponentsForReservedPath(src);
     writeLock();
     try {
       checkOperation(OperationCategory.WRITE);
       checkNameNodeSafeMode("Cannot set ACL on " + src);
-      src = dir.resolvePath(pc, src, pathComponents);
-      final INodesInPath iip = dir.getINodesInPath4Write(src);
-      dir.checkOwner(pc, iip);
-      List<AclEntry> newAcl = dir.setAcl(src, aclSpec);
-      getEditLog().logSetAcl(src, newAcl);
-      resultingStat = getAuditFileInfo(src, false);
+      auditStat = FSDirAclOp.setAcl(dir, src, aclSpec);
     } catch (AccessControlException e) {
-      logAuditEvent(false, "setAcl", srcArg);
+      logAuditEvent(false, "setAcl", src);
       throw e;
     } finally {
       writeUnlock();
     }
     getEditLog().logSync();
-    logAuditEvent(true, "setAcl", srcArg, null, resultingStat);
+    logAuditEvent(true, "setAcl", src, null, auditStat);
   }
 
   AclStatus getAclStatus(String src) throws IOException {
-    checkAclsConfigFlag();
-    FSPermissionChecker pc = getPermissionChecker();
     checkOperation(OperationCategory.READ);
-    byte[][] pathComponents = FSDirectory.getPathComponentsForReservedPath(src);
     boolean success = false;
     readLock();
     try {
       checkOperation(OperationCategory.READ);
-      src = dir.resolvePath(pc, src, pathComponents);
-      INodesInPath iip = dir.getINodesInPath(src, true);
-      if (isPermissionEnabled) {
-        dir.checkPermission(pc, iip, false, null, null, null, null);
-      }
-      final AclStatus ret = dir.getAclStatus(src);
+      final AclStatus ret = FSDirAclOp.getAclStatus(dir, src);
       success = true;
       return ret;
     } finally {
@@ -8369,15 +8311,6 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     }
   }
 
-  private void checkAclsConfigFlag() throws AclException {
-    if (!aclsEnabled) {
-      throw new AclException(String.format(
-          "The ACL operation has been rejected.  "
-              + "Support for ACLs has been disabled by setting %s to false.",
-          DFSConfigKeys.DFS_NAMENODE_ACLS_ENABLED_KEY));
-    }
-  }
-
   private void checkXAttrsConfigFlag() throws IOException {
     if (!xattrsEnabled) {
       throw new IOException(String.format(

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9297f980/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAuditLogger.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAuditLogger.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAuditLogger.java
index c91cd75..0c119bf 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAuditLogger.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAuditLogger.java
@@ -18,21 +18,6 @@
 
 package org.apache.hadoop.hdfs.server.namenode;
 
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_AUDIT_LOGGERS_KEY;
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_ACLS_ENABLED_KEY;
-import static org.apache.hadoop.hdfs.DFSConfigKeys.NNTOP_ENABLED_KEY;
-import static org.apache.hadoop.test.GenericTestUtils.assertExceptionContains;
-import static org.junit.Assert.*;
-import static org.mockito.Matchers.anyListOf;
-import static org.mockito.Matchers.anyString;
-
-import java.io.IOException;
-import java.net.HttpURLConnection;
-import java.net.InetAddress;
-import java.net.URI;
-import java.net.URISyntaxException;
-import java.util.List;
-
 import com.google.common.collect.Lists;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileStatus;
@@ -53,6 +38,22 @@ import org.junit.Before;
 import org.junit.Test;
 import org.mockito.Mockito;
 
+import java.io.IOException;
+import java.net.HttpURLConnection;
+import java.net.InetAddress;
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.util.List;
+
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_ACLS_ENABLED_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_AUDIT_LOGGERS_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.NNTOP_ENABLED_KEY;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+import static org.mockito.Mockito.doThrow;
+
 /**
  * Tests for the {@link AuditLogger} custom audit logging interface.
  */
@@ -208,27 +209,10 @@ public class TestAuditLogger {
     try {
       cluster.waitClusterUp();
       final FSDirectory dir = cluster.getNamesystem().getFSDirectory();
-      // Set up mock FSDirectory to test FSN audit logging during failure
+
       final FSDirectory mockedDir = Mockito.spy(dir);
-      Mockito.doThrow(new AccessControlException("mock setAcl exception")).
-          when(mockedDir).
-          setAcl(anyString(), anyListOf(AclEntry.class));
-      Mockito.doThrow(new AccessControlException("mock getAclStatus exception")).
-          when(mockedDir).
-          getAclStatus(anyString());
-      Mockito.doThrow(new AccessControlException("mock removeAcl exception")).
-          when(mockedDir).
-          removeAcl(anyString());
-      Mockito.doThrow(new AccessControlException("mock removeDefaultAcl exception")).
-          when(mockedDir).
-          removeDefaultAcl(anyString());
-      Mockito.doThrow(new AccessControlException("mock removeAclEntries exception")).
-          when(mockedDir).
-          removeAclEntries(anyString(), anyListOf(AclEntry.class));
-      Mockito.doThrow(new AccessControlException("mock modifyAclEntries exception")).
-          when(mockedDir).
-          modifyAclEntries(anyString(), anyListOf(AclEntry.class));
-      // Replace the FSD with the mock FSD.
+      AccessControlException ex = new AccessControlException();
+      doThrow(ex).when(mockedDir).getPermissionChecker();
       cluster.getNamesystem().setFSDirectory(mockedDir);
       assertTrue(DummyAuditLogger.initialized);
       DummyAuditLogger.resetLogCount();
@@ -239,39 +223,28 @@ public class TestAuditLogger {
 
       try {
         fs.getAclStatus(p);
-      } catch (AccessControlException e) {
-        assertExceptionContains("mock getAclStatus exception", e);
-      }
+      } catch (AccessControlException ignored) {}
 
       try {
         fs.setAcl(p, acls);
-      } catch (AccessControlException e) {
-        assertExceptionContains("mock setAcl exception", e);
-      }
+      } catch (AccessControlException ignored) {}
 
       try {
         fs.removeAcl(p);
-      } catch (AccessControlException e) {
-        assertExceptionContains("mock removeAcl exception", e);
-      }
+      } catch (AccessControlException ignored) {}
 
       try {
         fs.removeDefaultAcl(p);
-      } catch (AccessControlException e) {
-        assertExceptionContains("mock removeDefaultAcl exception", e);
-      }
+      } catch (AccessControlException ignored) {}
 
       try {
         fs.removeAclEntries(p, acls);
-      } catch (AccessControlException e) {
-        assertExceptionContains("mock removeAclEntries exception", e);
-      }
+      } catch (AccessControlException ignored) {}
 
       try {
         fs.modifyAclEntries(p, acls);
-      } catch (AccessControlException e) {
-        assertExceptionContains("mock modifyAclEntries exception", e);
-      }
+      } catch (AccessControlException ignored) {}
+
       assertEquals(6, DummyAuditLogger.logCount);
       assertEquals(6, DummyAuditLogger.unsuccessfulCount);
     } finally {


[23/29] hadoop git commit: YARN-2869. CapacityScheduler should trim sub queue names when parse configuration. Contributed by Wangda Tan

Posted by vi...@apache.org.
YARN-2869. CapacityScheduler should trim sub queue names when parse configuration. Contributed by Wangda Tan


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

Branch: refs/heads/HDFS-EC
Commit: e69af836f34f16fba565ab112c9bf0d367675b16
Parents: 475c6b4
Author: Jian He <ji...@apache.org>
Authored: Fri Dec 5 17:33:39 2014 -0800
Committer: Jian He <ji...@apache.org>
Committed: Fri Dec 5 17:33:39 2014 -0800

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                 |   3 +
 .../CapacitySchedulerConfiguration.java         |  10 +-
 .../scheduler/capacity/TestQueueParsing.java    | 110 +++++++++++++++++++
 3 files changed, 122 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/e69af836/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index 0b88959..0d7a843 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -194,6 +194,9 @@ Release 2.7.0 - UNRELEASED
     YARN-2461. Fix PROCFS_USE_SMAPS_BASED_RSS_ENABLED property in
     YarnConfiguration. (rchiang via rkanter)
 
+    YARN-2869. CapacityScheduler should trim sub queue names when parse
+    configuration. (Wangda Tan via jianhe)
+
 Release 2.6.0 - 2014-11-18
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e69af836/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacitySchedulerConfiguration.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacitySchedulerConfiguration.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacitySchedulerConfiguration.java
index 0a49224..5bbb436 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacitySchedulerConfiguration.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacitySchedulerConfiguration.java
@@ -260,7 +260,7 @@ public class CapacitySchedulerConfiguration extends ReservationSchedulerConfigur
     }
   }
 
-  private String getQueuePrefix(String queue) {
+  static String getQueuePrefix(String queue) {
     String queueName = PREFIX + queue + DOT;
     return queueName;
   }
@@ -538,6 +538,14 @@ public class CapacitySchedulerConfiguration extends ReservationSchedulerConfigur
   public String[] getQueues(String queue) {
     LOG.debug("CSConf - getQueues called for: queuePrefix=" + getQueuePrefix(queue));
     String[] queues = getStrings(getQueuePrefix(queue) + QUEUES);
+    List<String> trimmedQueueNames = new ArrayList<String>();
+    if (null != queues) {
+      for (String s : queues) {
+        trimmedQueueNames.add(s.trim());
+      }
+      queues = trimmedQueueNames.toArray(new String[0]);
+    }
+ 
     LOG.debug("CSConf - getQueues: queuePrefix=" + getQueuePrefix(queue) + 
         ", queues=" + ((queues == null) ? "" : StringUtils.arrayToString(queues)));
     return queues;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e69af836/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestQueueParsing.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestQueueParsing.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestQueueParsing.java
index cf2e5ce..5a9fbe1 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestQueueParsing.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestQueueParsing.java
@@ -82,6 +82,56 @@ public class TestQueueParsing {
     ServiceOperations.stopQuietly(capacityScheduler);
   }
   
+  private void setupQueueConfigurationWithSpacesShouldBeTrimmed(
+      CapacitySchedulerConfiguration conf) {
+    // Define top-level queues
+    conf.set(
+        CapacitySchedulerConfiguration
+            .getQueuePrefix(CapacitySchedulerConfiguration.ROOT)
+            + CapacitySchedulerConfiguration.QUEUES, " a ,b, c");
+
+    final String A = CapacitySchedulerConfiguration.ROOT + ".a";
+    conf.setCapacity(A, 10);
+    conf.setMaximumCapacity(A, 15);
+    
+    final String B = CapacitySchedulerConfiguration.ROOT + ".b";
+    conf.setCapacity(B, 20);
+    
+    final String C = CapacitySchedulerConfiguration.ROOT + ".c";
+    conf.setCapacity(C, 70);
+    conf.setMaximumCapacity(C, 70);
+  }
+  
+  private void setupNestedQueueConfigurationWithSpacesShouldBeTrimmed(
+      CapacitySchedulerConfiguration conf) {
+    // Define top-level queues
+    conf.set(
+        CapacitySchedulerConfiguration
+            .getQueuePrefix(CapacitySchedulerConfiguration.ROOT)
+            + CapacitySchedulerConfiguration.QUEUES, " a ,b, c");
+
+    final String A = CapacitySchedulerConfiguration.ROOT + ".a";
+    conf.setCapacity(A, 10);
+    conf.setMaximumCapacity(A, 15);
+
+    final String B = CapacitySchedulerConfiguration.ROOT + ".b";
+    conf.setCapacity(B, 20);
+
+    final String C = CapacitySchedulerConfiguration.ROOT + ".c";
+    conf.setCapacity(C, 70);
+    conf.setMaximumCapacity(C, 70);
+
+    // sub queues for A
+    conf.set(CapacitySchedulerConfiguration.getQueuePrefix(A)
+        + CapacitySchedulerConfiguration.QUEUES, "a1, a2 ");
+
+    final String A1 = CapacitySchedulerConfiguration.ROOT + ".a.a1";
+    conf.setCapacity(A1, 60);
+
+    final String A2 = CapacitySchedulerConfiguration.ROOT + ".a.a2";
+    conf.setCapacity(A2, 40);
+  }
+  
   private void setupQueueConfiguration(CapacitySchedulerConfiguration conf) {
     
     // Define top-level queues
@@ -659,4 +709,64 @@ public class TestQueueParsing {
         DELTA);
     capacityScheduler.stop();
   }
+  
+  @Test
+  public void testQueueParsingShouldTrimSpaces() throws Exception {
+    CapacitySchedulerConfiguration csConf = 
+      new CapacitySchedulerConfiguration();
+    setupQueueConfigurationWithSpacesShouldBeTrimmed(csConf);
+    YarnConfiguration conf = new YarnConfiguration(csConf);
+
+    CapacityScheduler capacityScheduler = new CapacityScheduler();
+    capacityScheduler.setConf(conf);
+    capacityScheduler.setRMContext(TestUtils.getMockRMContext());
+    capacityScheduler.init(conf);
+    capacityScheduler.start();
+    capacityScheduler.reinitialize(conf, TestUtils.getMockRMContext());
+    
+    CSQueue a = capacityScheduler.getQueue("a");
+    Assert.assertNotNull(a);
+    Assert.assertEquals(0.10, a.getAbsoluteCapacity(), DELTA);
+    Assert.assertEquals(0.15, a.getAbsoluteMaximumCapacity(), DELTA);
+    
+    CSQueue c = capacityScheduler.getQueue("c");
+    Assert.assertNotNull(c);
+    Assert.assertEquals(0.70, c.getAbsoluteCapacity(), DELTA);
+    Assert.assertEquals(0.70, c.getAbsoluteMaximumCapacity(), DELTA);
+  }
+  
+  @Test
+  public void testNestedQueueParsingShouldTrimSpaces() throws Exception {
+    CapacitySchedulerConfiguration csConf = 
+      new CapacitySchedulerConfiguration();
+    setupNestedQueueConfigurationWithSpacesShouldBeTrimmed(csConf);
+    YarnConfiguration conf = new YarnConfiguration(csConf);
+
+    CapacityScheduler capacityScheduler = new CapacityScheduler();
+    capacityScheduler.setConf(conf);
+    capacityScheduler.setRMContext(TestUtils.getMockRMContext());
+    capacityScheduler.init(conf);
+    capacityScheduler.start();
+    capacityScheduler.reinitialize(conf, TestUtils.getMockRMContext());
+    
+    CSQueue a = capacityScheduler.getQueue("a");
+    Assert.assertNotNull(a);
+    Assert.assertEquals(0.10, a.getAbsoluteCapacity(), DELTA);
+    Assert.assertEquals(0.15, a.getAbsoluteMaximumCapacity(), DELTA);
+    
+    CSQueue c = capacityScheduler.getQueue("c");
+    Assert.assertNotNull(c);
+    Assert.assertEquals(0.70, c.getAbsoluteCapacity(), DELTA);
+    Assert.assertEquals(0.70, c.getAbsoluteMaximumCapacity(), DELTA);
+    
+    CSQueue a1 = capacityScheduler.getQueue("a1");
+    Assert.assertNotNull(a1);
+    Assert.assertEquals(0.10 * 0.6, a1.getAbsoluteCapacity(), DELTA);
+    Assert.assertEquals(0.15, a1.getAbsoluteMaximumCapacity(), DELTA);
+    
+    CSQueue a2 = capacityScheduler.getQueue("a2");
+    Assert.assertNotNull(a2);
+    Assert.assertEquals(0.10 * 0.4, a2.getAbsoluteCapacity(), DELTA);
+    Assert.assertEquals(0.15, a2.getAbsoluteMaximumCapacity(), DELTA);
+  }
 }