Переглянути джерело

Merging r1534279 through r1534706 from trunk to branch HDFS-2832

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/HDFS-2832@1534707 13f79535-47bb-0310-9956-ffa450edef68
Arpit Agarwal 11 роки тому
батько
коміт
ef17685f79
38 змінених файлів з 1227 додано та 232 видалено
  1. 3 0
      hadoop-common-project/hadoop-common/CHANGES.txt
  2. 1 1
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/filter/AbstractPatternFilter.java
  3. 1 1
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/package-info.java
  4. 53 10
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/metrics2/filter/TestPatternFilter.java
  5. 138 0
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/metrics2/sink/TestFileSink.java
  6. 11 0
      hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
  7. 2 0
      hadoop-hdfs-project/hadoop-hdfs/pom.xml
  8. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
  9. 7 8
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementPolicy.java
  10. 6 6
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementPolicyDefault.java
  11. 42 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementStatus.java
  12. 44 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementStatusDefault.java
  13. 9 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/web/resources/DatanodeWebHdfsMethods.java
  14. 6 6
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java
  15. 3 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/dfshealth.dust.html
  16. 1 31
      hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/dfshealth.js
  17. 13 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/explorer-block-info.dust.html
  18. 26 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/explorer.dust.html
  19. 86 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/explorer.html
  20. 182 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/explorer.js
  21. 258 0
      hadoop-hdfs-project/hadoop-hdfs/src/site/apt/HdfsNfsGateway.apt.vm
  22. 74 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFsck.java
  23. 1 0
      hadoop-project/src/site/site.xml
  24. 9 2
      hadoop-yarn-project/CHANGES.txt
  25. 2 3
      hadoop-yarn-project/hadoop-yarn/bin/yarn.cmd
  26. 0 8
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/Queue.java
  27. 0 5
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/LeafQueue.java
  28. 0 5
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/ParentQueue.java
  29. 1 4
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSLeafQueue.java
  30. 1 10
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSQueue.java
  31. 5 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FairSchedulerConfiguration.java
  32. 13 24
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/QueueManager.java
  33. 0 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fifo/FifoScheduler.java
  34. 27 65
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/QueueACLsTestBase.java
  35. 73 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacitySchedulerQueueACLs.java
  36. 39 30
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFairScheduler.java
  37. 62 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFairSchedulerQueueACLs.java
  38. 27 4
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/apt/FairScheduler.apt.vm

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

@@ -366,6 +366,9 @@ Release 2.3.0 - UNRELEASED
     HDFS-5276. FileSystem.Statistics should use thread-local counters to avoid
     multi-threaded performance issues on read/write.  (Colin Patrick McCabe)
 
+    HADOOP-9291. enhance unit-test coverage of package o.a.h.metrics2 (Ivan A.
+    Veselovsky via jeagles)
+
   OPTIMIZATIONS
 
     HADOOP-9748. Reduce blocking on UGI.ensureInitialized (daryn)

+ 1 - 1
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/filter/AbstractPatternFilter.java

@@ -112,7 +112,7 @@ public abstract class AbstractPatternFilter extends MetricsFilter {
       return false;
     }
     // Reject if no match in whitelist only mode
-    if (ipat != null && epat == null) {
+    if (!includeTagPatterns.isEmpty() && excludeTagPatterns.isEmpty()) {
       return false;
     }
     return true;

+ 1 - 1
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/package-info.java

@@ -234,7 +234,7 @@
     patterns.
   </p>
   <p>Similarly, you can specify the <code>record.filter</code> and
-    <code>metrics.filter</code> options, which operate at record and metric
+    <code>metric.filter</code> options, which operate at record and metric
     level, respectively. Filters can be combined to optimize
     the filtering efficiency.</p>
 

+ 53 - 10
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/metrics2/filter/TestPatternFilter.java

@@ -23,9 +23,11 @@ import java.util.List;
 
 import org.apache.commons.configuration.SubsetConfiguration;
 import org.junit.Test;
+
 import static org.junit.Assert.*;
 import static org.mockito.Mockito.*;
 
+import org.apache.hadoop.metrics2.MetricsFilter;
 import org.apache.hadoop.metrics2.MetricsRecord;
 import org.apache.hadoop.metrics2.MetricsTag;
 import org.apache.hadoop.metrics2.impl.ConfigBuilder;
@@ -53,7 +55,7 @@ public class TestPatternFilter {
         .add("p.include.tags", "foo:f").subset("p");
     shouldAccept(wl, "foo");
     shouldAccept(wl, Arrays.asList(tag("bar", "", ""),
-                                   tag("foo", "", "f")));
+                                   tag("foo", "", "f")), new boolean[] {false, true});
     shouldAccept(wl, mockMetricsRecord("foo", Arrays.asList(
       tag("bar", "", ""), tag("foo", "", "f"))));
     shouldReject(wl, "bar");
@@ -78,7 +80,7 @@ public class TestPatternFilter {
       tag("bar", "", ""))));
     shouldReject(bl, "foo");
     shouldReject(bl, Arrays.asList(tag("bar", "", ""),
-                                   tag("foo", "", "f")));
+                                   tag("foo", "", "f")), new boolean[] {true, false});
     shouldReject(bl, mockMetricsRecord("foo", Arrays.asList(
       tag("bar", "", ""))));
     shouldReject(bl, mockMetricsRecord("bar", Arrays.asList(
@@ -125,15 +127,61 @@ public class TestPatternFilter {
     shouldAccept(c, mockMetricsRecord("foo", Arrays.asList(
       tag("foo", "", "f"))));
   }
-
+  
   static void shouldAccept(SubsetConfiguration conf, String s) {
     assertTrue("accepts "+ s, newGlobFilter(conf).accepts(s));
     assertTrue("accepts "+ s, newRegexFilter(conf).accepts(s));
   }
 
+  // Version for one tag:
   static void shouldAccept(SubsetConfiguration conf, List<MetricsTag> tags) {
-    assertTrue("accepts "+ tags, newGlobFilter(conf).accepts(tags));
-    assertTrue("accepts "+ tags, newRegexFilter(conf).accepts(tags));
+    shouldAcceptImpl(true, conf, tags, new boolean[] {true});
+  }
+  // Version for multiple tags: 
+  static void shouldAccept(SubsetConfiguration conf, List<MetricsTag> tags, 
+      boolean[] expectedAcceptedSpec) {
+    shouldAcceptImpl(true, conf, tags, expectedAcceptedSpec);
+  }
+
+  // Version for one tag:
+  static void shouldReject(SubsetConfiguration conf, List<MetricsTag> tags) {
+    shouldAcceptImpl(false, conf, tags, new boolean[] {false});
+  }
+  // Version for multiple tags: 
+  static void shouldReject(SubsetConfiguration conf, List<MetricsTag> tags, 
+      boolean[] expectedAcceptedSpec) {
+    shouldAcceptImpl(false, conf, tags, expectedAcceptedSpec);
+  }
+  
+  private static void shouldAcceptImpl(final boolean expectAcceptList,  
+      SubsetConfiguration conf, List<MetricsTag> tags, boolean[] expectedAcceptedSpec) {
+    final MetricsFilter globFilter = newGlobFilter(conf);
+    final MetricsFilter regexFilter = newRegexFilter(conf);
+    
+    // Test acceptance of the tag list:  
+    assertEquals("accepts "+ tags, expectAcceptList, globFilter.accepts(tags));
+    assertEquals("accepts "+ tags, expectAcceptList, regexFilter.accepts(tags));
+    
+    // Test results on each of the individual tags:
+    int acceptedCount = 0;
+    for (int i=0; i<tags.size(); i++) {
+      MetricsTag tag = tags.get(i);
+      boolean actGlob = globFilter.accepts(tag);
+      boolean actRegex = regexFilter.accepts(tag);
+      assertEquals("accepts "+tag, expectedAcceptedSpec[i], actGlob);
+      // Both the filters should give the same result:
+      assertEquals(actGlob, actRegex);
+      if (actGlob) {
+        acceptedCount++;
+      }
+    }
+    if (expectAcceptList) {
+      // At least one individual tag should be accepted:
+      assertTrue("No tag of the following accepted: " + tags, acceptedCount > 0);
+    } else {
+      // At least one individual tag should be rejected: 
+      assertTrue("No tag of the following rejected: " + tags, acceptedCount < tags.size());
+    }
   }
 
   /**
@@ -152,11 +200,6 @@ public class TestPatternFilter {
     assertTrue("rejects "+ s, !newRegexFilter(conf).accepts(s));
   }
 
-  static void shouldReject(SubsetConfiguration conf, List<MetricsTag> tags) {
-    assertTrue("rejects "+ tags, !newGlobFilter(conf).accepts(tags));
-    assertTrue("rejects "+ tags, !newRegexFilter(conf).accepts(tags));
-  }
-
   /**
    * Asserts that filters with the given configuration reject the given record.
    * 

+ 138 - 0
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/metrics2/sink/TestFileSink.java

@@ -0,0 +1,138 @@
+/*
+ * 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.metrics2.sink;
+
+import java.io.ByteArrayOutputStream;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.regex.Pattern;
+
+import org.apache.hadoop.io.IOUtils;
+import org.apache.hadoop.metrics2.MetricsSystem;
+import org.apache.hadoop.metrics2.annotation.Metric;
+import org.apache.hadoop.metrics2.annotation.Metrics;
+import org.apache.hadoop.metrics2.annotation.Metric.Type;
+import org.apache.hadoop.metrics2.impl.ConfigBuilder;
+import org.apache.hadoop.metrics2.impl.MetricsSystemImpl;
+import org.apache.hadoop.metrics2.impl.TestMetricsConfig;
+import org.apache.hadoop.metrics2.lib.MutableGaugeInt;
+import org.junit.After;
+import org.junit.Test;
+import static org.junit.Assert.*;
+
+public class TestFileSink {
+  
+  private File outFile;
+
+  // The 2 sample metric classes:
+  @Metrics(name="testRecord1", context="test1")
+  static class MyMetrics1 {
+    @Metric(value={"testTag1", ""}, type=Type.TAG) 
+    String testTag1() { return "testTagValue1"; }
+    
+    @Metric(value={"testTag2", ""}, type=Type.TAG) 
+    String gettestTag2() { return "testTagValue2"; }
+    
+    @Metric(value={"testMetric1", "An integer gauge"},always=true) 
+    MutableGaugeInt testMetric1;
+    
+    @Metric(value={"testMetric2", "An integer gauge"},always=true) 
+    MutableGaugeInt testMetric2;
+
+    public MyMetrics1 registerWith(MetricsSystem ms) {
+      return ms.register("m1", null, this);
+    }
+  }
+  
+  @Metrics(name="testRecord2", context="test1")
+  static class MyMetrics2 {
+    @Metric(value={"testTag22", ""}, type=Type.TAG) 
+    String testTag1() { return "testTagValue22"; }
+
+    public MyMetrics2 registerWith(MetricsSystem ms) {
+      return ms.register("m2", null, this);
+    }
+  }
+  
+  private File getTestTempFile(String prefix, String suffix) throws IOException {
+    String tmpPath = System.getProperty("java.io.tmpdir", "/tmp");
+    String user = System.getProperty("user.name", "unknown-user");
+    File dir = new File(tmpPath + "/" + user);
+    dir.mkdirs();
+    return File.createTempFile(prefix, suffix, dir);
+  }
+  
+  @Test(timeout=6000) 
+  public void testFileSink() throws IOException {
+    outFile = getTestTempFile("test-file-sink-", ".out");
+    final String outPath = outFile.getAbsolutePath();  
+    
+    // NB: specify large period to avoid multiple metrics snapshotting: 
+    new ConfigBuilder().add("*.period", 10000)
+        .add("test.sink.mysink0.class", FileSink.class.getName())
+        .add("test.sink.mysink0.filename", outPath)
+        // NB: we filter by context to exclude "metricssystem" context metrics:
+        .add("test.sink.mysink0.context", "test1")
+        .save(TestMetricsConfig.getTestFilename("hadoop-metrics2-test"));
+    MetricsSystemImpl ms = new MetricsSystemImpl("test");
+    ms.start();
+
+    final MyMetrics1 mm1 
+      = new MyMetrics1().registerWith(ms);
+    new MyMetrics2().registerWith(ms);
+
+    mm1.testMetric1.incr();
+    mm1.testMetric2.incr(2);
+
+    ms.publishMetricsNow(); // publish the metrics
+    ms.stop();
+    ms.shutdown();
+    
+    InputStream is = new FileInputStream(outFile);
+    ByteArrayOutputStream baos = new ByteArrayOutputStream((int)outFile.length());
+    IOUtils.copyBytes(is, baos, 1024, true);
+    String outFileContent = new String(baos.toByteArray(), "UTF-8");
+
+    // Check the out file content. Should be something like the following:
+    //1360244820087 test1.testRecord1: Context=test1, testTag1=testTagValue1, testTag2=testTagValue2, Hostname=myhost, testMetric1=1, testMetric2=2
+    //1360244820089 test1.testRecord2: Context=test1, testTag22=testTagValue22, Hostname=myhost
+    
+    // Note that in the below expression we allow tags and metrics to go in arbitrary order.  
+    Pattern expectedContentPattern = Pattern.compile(
+        // line #1:
+        "^\\d+\\s+test1.testRecord1:\\s+Context=test1,\\s+" +
+        "(testTag1=testTagValue1,\\s+testTag2=testTagValue2|testTag2=testTagValue2,\\s+testTag1=testTagValue1)," +
+        "\\s+Hostname=.*,\\s+(testMetric1=1,\\s+testMetric2=2|testMetric2=2,\\s+testMetric1=1)" +
+        // line #2:
+        "$[\\n\\r]*^\\d+\\s+test1.testRecord2:\\s+Context=test1," +
+        "\\s+testTag22=testTagValue22,\\s+Hostname=.*$[\\n\\r]*", 
+         Pattern.MULTILINE);
+     assertTrue(expectedContentPattern.matcher(outFileContent).matches());
+  }
+  
+  @After
+  public void after() {
+    if (outFile != null) {
+      outFile.delete();
+      assertTrue(!outFile.exists());
+    }
+  }
+}

+ 11 - 0
hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt

@@ -261,6 +261,9 @@ Release 2.3.0 - UNRELEASED
     HDFS-5379. Update links to datanode information in dfshealth.html. (Haohui
     Mai via jing9)
 
+    HDFS-5382. Implement the UI of browsing filesystems in HTML 5 page. (Haohui
+    Mai via jing9)
+
   IMPROVEMENTS
 
     HDFS-5267. Remove volatile from LightWeightHashSet. (Junping Du via llu)
@@ -329,6 +332,9 @@ Release 2.3.0 - UNRELEASED
     HDFS-4511. Cover package org.apache.hadoop.hdfs.tools with unit test
     (Andrey Klochkov via jeagles)
 
+    HDFS-4885. Improve the verifyBlockPlacement() API in BlockPlacementPolicy.
+    (Junping Du via szetszwo)
+
   OPTIMIZATIONS
 
     HDFS-5239.  Allow FSNamesystem lock fairness to be configurable (daryn)
@@ -369,6 +375,9 @@ Release 2.3.0 - UNRELEASED
     HDFS-5336. DataNode should not output 'StartupProgress' metrics.
     (Akira Ajisaka via cnauroth)
 
+    HDFS-5400.  DFS_CLIENT_MMAP_CACHE_THREAD_RUNS_PER_TIMEOUT constant is set
+    to the wrong value.  (Colin Patrick McCabe)
+
 Release 2.2.1 - UNRELEASED
 
   INCOMPATIBLE CHANGES
@@ -419,6 +428,8 @@ Release 2.2.1 - UNRELEASED
     and range in error message. (Kousuke Saruta via suresh)
 
     HDFS-5365. Fix libhdfs compile error on FreeBSD9. (Radim Kolar via cnauroth)
+    
+    HDFS-5347. Add HDFS NFS user guide. (brandonli)
 
 Release 2.2.0 - 2013-10-13
 

+ 2 - 0
hadoop-hdfs-project/hadoop-hdfs/pom.xml

@@ -550,6 +550,8 @@ http://maven.apache.org/xsd/maven-4.0.0.xsd">
             <exclude>src/main/webapps/static/dust-full-2.0.0.min.js</exclude>
             <exclude>src/main/webapps/static/dust-helpers-1.1.1.min.js</exclude>
             <exclude>src/main/webapps/hdfs/dfshealth.dust.html</exclude>
+            <exclude>src/main/webapps/hdfs/explorer-block-info.dust.html</exclude>
+            <exclude>src/main/webapps/hdfs/explorer.dust.html</exclude>
           </excludes>
         </configuration>
       </plugin>

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

@@ -383,7 +383,7 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
   public static final int DFS_CLIENT_MMAP_CACHE_SIZE_DEFAULT = 1024;
   public static final String DFS_CLIENT_MMAP_CACHE_TIMEOUT_MS = "dfs.client.mmap.cache.timeout.ms";
   public static final long DFS_CLIENT_MMAP_CACHE_TIMEOUT_MS_DEFAULT  = 15 * 60 * 1000;
-  public static final String DFS_CLIENT_MMAP_CACHE_THREAD_RUNS_PER_TIMEOUT = "dfs.client.mmap.cache.timeout.ms";
+  public static final String DFS_CLIENT_MMAP_CACHE_THREAD_RUNS_PER_TIMEOUT = "dfs.client.mmap.cache.thread.runs.per.timeout";
   public static final int DFS_CLIENT_MMAP_CACHE_THREAD_RUNS_PER_TIMEOUT_DEFAULT  = 4;
 
   // property for fsimage compression

+ 7 - 8
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementPolicy.java

@@ -100,18 +100,17 @@ public abstract class BlockPlacementPolicy {
   }
 
   /**
-   * Verify that the block is replicated on at least minRacks different racks
-   * if there is more than minRacks rack in the system.
+   * Verify if the block's placement meets requirement of placement policy,
+   * i.e. replicas are placed on no less than minRacks racks in the system.
    * 
    * @param srcPath the full pathname of the file to be verified
    * @param lBlk block with locations
-   * @param minRacks number of racks the block should be replicated to
-   * @return the difference between the required and the actual number of racks
-   * the block is replicated to.
+   * @param numOfReplicas replica number of file to be verified
+   * @return the result of verification
    */
-  abstract public int verifyBlockPlacement(String srcPath,
-                                           LocatedBlock lBlk,
-                                           int minRacks);
+  abstract public BlockPlacementStatus verifyBlockPlacement(String srcPath,
+      LocatedBlock lBlk,
+      int numOfReplicas);
   /**
    * Decide whether deleting the specified replica of the block still makes 
    * the block conform to the configured block placement policy.

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

@@ -698,22 +698,22 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy {
   }
 
   @Override
-  public int verifyBlockPlacement(String srcPath,
-                                  LocatedBlock lBlk,
-                                  int minRacks) {
+  public BlockPlacementStatus verifyBlockPlacement(String srcPath,
+      LocatedBlock lBlk, int numberOfReplicas) {
     DatanodeInfo[] locs = lBlk.getLocations();
     if (locs == null)
       locs = DatanodeDescriptor.EMPTY_ARRAY;
     int numRacks = clusterMap.getNumOfRacks();
     if(numRacks <= 1) // only one rack
-      return 0;
-    minRacks = Math.min(minRacks, numRacks);
+      return new BlockPlacementStatusDefault(
+          Math.min(numRacks, numberOfReplicas), numRacks);
+    int minRacks = Math.min(2, numberOfReplicas);
     // 1. Check that all locations are different.
     // 2. Count locations on different racks.
     Set<String> racks = new TreeSet<String>();
     for (DatanodeInfo dn : locs)
       racks.add(dn.getNetworkLocation());
-    return minRacks - racks.size();
+    return new BlockPlacementStatusDefault(racks.size(), minRacks);
   }
 
   @Override

+ 42 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementStatus.java

@@ -0,0 +1,42 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.server.blockmanagement;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public interface BlockPlacementStatus {
+
+  /**
+   * Boolean value to identify if replicas of this block satisfy requirement of 
+   * placement policy
+   * @return if replicas satisfy placement policy's requirement 
+   */
+  public boolean isPlacementPolicySatisfied();
+  
+  /**
+   * Get description info for log or printed in case replicas are failed to meet
+   * requirement of placement policy
+   * @return description in case replicas are failed to meet requirement of
+   * placement policy
+   */
+  public String getErrorDescription();
+
+}

+ 44 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementStatusDefault.java

@@ -0,0 +1,44 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.server.blockmanagement;
+
+public class BlockPlacementStatusDefault implements BlockPlacementStatus {
+
+  private int requiredRacks = 0;
+  private int currentRacks = 0;
+  
+  public BlockPlacementStatusDefault(int currentRacks, int requiredRacks){
+    this.requiredRacks = requiredRacks;
+    this.currentRacks = currentRacks;
+  }
+  
+  @Override
+  public boolean isPlacementPolicySatisfied() {
+    return requiredRacks <= currentRacks;
+  }
+
+  @Override
+  public String getErrorDescription() {
+    if (isPlacementPolicySatisfied()) {
+      return null;
+    }
+    return "Block should be additionally replicated on " + 
+        (requiredRacks - currentRacks) + " more rack(s).";
+  }
+
+}

+ 9 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/web/resources/DatanodeWebHdfsMethods.java

@@ -413,8 +413,15 @@ public class DatanodeWebHdfsMethods {
       final long n = length.getValue() != null ?
         Math.min(length.getValue(), in.getVisibleLength() - offset.getValue()) :
         in.getVisibleLength() - offset.getValue();
-      return Response.ok(new OpenEntity(in, n, dfsclient)).type(
-          MediaType.APPLICATION_OCTET_STREAM).build();
+
+      /**
+       * Allow the Web UI to perform an AJAX request to get the data.
+       */
+      return Response.ok(new OpenEntity(in, n, dfsclient))
+          .type(MediaType.APPLICATION_OCTET_STREAM)
+          .header("Access-Control-Allow-Methods", "GET")
+          .header("Access-Control-Allow-Origin", "*")
+          .build();
     }
     case GETFILECHECKSUM:
     {

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

@@ -50,6 +50,7 @@ import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockPlacementPolicy;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockPlacementStatus;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.datanode.CachingStrategy;
 import org.apache.hadoop.net.NetUtils;
@@ -374,9 +375,10 @@ public class NamenodeFsck {
                     locs.length + " replica(s).");
       }
       // verify block placement policy
-      int missingRacks = BlockPlacementPolicy.getInstance(conf, null, networktopology).
-                           verifyBlockPlacement(path, lBlk, Math.min(2,targetFileReplication));
-      if (missingRacks > 0) {
+      BlockPlacementStatus blockPlacementStatus = 
+          BlockPlacementPolicy.getInstance(conf, null, networktopology).
+              verifyBlockPlacement(path, lBlk, targetFileReplication);
+      if (!blockPlacementStatus.isPlacementPolicySatisfied()) {
         res.numMisReplicatedBlocks++;
         misReplicatedPerFile++;
         if (!showFiles) {
@@ -385,9 +387,7 @@ public class NamenodeFsck {
           out.print(path + ": ");
         }
         out.println(" Replica placement policy is violated for " + 
-                    block +
-                    ". Block should be additionally replicated on " + 
-                    missingRacks + " more rack(s).");
+                    block + ". " + blockPlacementStatus.getErrorDescription());
       }
       report.append(i + ". " + blkName + " len=" + block.getNumBytes());
       if (locs.length == 0) {

+ 3 - 3
hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/dfshealth.dust.html

@@ -47,7 +47,7 @@
   </div>
 </div>
 
-<a id="browse-dir-first" style="cursor:pointer">Browse the filesystem</a>  <a href="/logs/">NameNode Logs</a>
+<a href="explorer.html">Browse the filesystem</a>  <a href="/logs/">NameNode Logs</a>
 
 <hr/>
 
@@ -56,7 +56,7 @@
   <div class="panel-body">
 
     <p>
-      Security is {#nnstat}{#SecurityModeEnabled}on{:else}off{/SecurityModeEnabled}{/nnstat}.</p>
+      Security is {#nnstat}{#SecurityEnabled}on{:else}off{/SecurityEnabled}{/nnstat}.</p>
     <p>{#nn}{#Safemode}{.}{:else}Safemode is off.{/Safemode}{/nn}</p>
 
     <p>
@@ -207,7 +207,7 @@
           </thead>
           {#nn.LiveNodes}
           <tr>
-            <td><a class="browse-dir-links" info-http-addr="{infoAddr}" info-https-addr="{infoSecureAddr}">{name}</a> ({xferaddr})</td>
+            <td>{name} ({xferaddr})</td>
             <td>{lastContact}</td>
             <td>{adminState}</td>
             <td>{capacity|fmt_bytes}</td>

+ 1 - 31
hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/dfshealth.js

@@ -19,19 +19,6 @@
   "use strict";
 
   var data = {};
-  function generate_browse_dn_link(info_http_addr, info_https_addr) {
-    var is_https = window.location.protocol === 'https:';
-    var authority = is_https ? info_https_addr : info_http_addr;
-
-    var nn_info_port = window.location.port;
-    if (nn_info_port === "") {
-      nn_info_port = is_https ? 443 : 80;
-    }
-
-    var l = '//' + authority + '/browseDirectory.jsp?dir=%2F&namenodeInfoPort=' +
-      nn_info_port + '&nnaddr=' + data.nnstat.HostAndPort;
-    return l;
-  }
 
   function render() {
     var helpers = {
@@ -56,24 +43,7 @@
 
     load_templates(dust, TEMPLATES, function() {
       dust.render('dfshealth', base.push(data), function(err, out) {
-
-        $('#panel').append(out);
-
-        $('#browse-dir-first').click(function () {
-          var len = data.nn.LiveNodes.length;
-          if (len < 1) {
-            show_err_msg('Cannot browse the DFS since there are no live nodes available.');
-            return false;
-          }
-
-          var dn = data.nn.LiveNodes[Math.floor(Math.random() * len)];
-          window.location.href = generate_browse_dn_link(dn.infoAddr, dn.infoSecureAddr);
-        });
-
-        $('.browse-dir-links').click(function () {
-          var http_addr = $(this).attr('info-http-addr'), https_addr = $(this).attr('info-https-addr');
-          window.location.href = generate_browse_dn_link(http_addr, https_addr);
-        });
+        $('#panel').html(out);
       });
     }, function () {
       show_err_msg('Failed to load the page.');

+ 13 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/explorer-block-info.dust.html

@@ -0,0 +1,13 @@
+{#block}
+<p>Block ID: {blockId}</p>
+<p>Block Pool ID: {blockPoolId}</p>
+<p>Generation Stamp: {generationStamp}</p>
+<p>Size: {numBytes}</p>
+{/block}
+<p>Availability:
+<ul>
+{#locations}
+<li>{hostName}</li>
+{/locations}
+</ul>
+</p>

+ 26 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/explorer.dust.html

@@ -0,0 +1,26 @@
+<table class="table">
+<thead>
+<tr>
+<th>Permission</th>
+<th>Owner</th>
+<th>Group</th>
+<th>Size</th>
+<th>Replication</th>
+<th>Block Size</th>
+<th>Name</th>
+</tr>
+</thead>
+<tbody>
+{#FileStatus}
+<tr>
+<td>{#helper_to_permission/}</td>
+<td>{owner}</td>
+<td>{group}</td>
+<td>{length|fmt_bytes}</td>
+<td>{replication}</td>
+<td>{blockSize|fmt_bytes}</td>
+<td><a style="cursor:pointer" inode-type="{type}" class="explorer-browse-links" inode-path="{pathSuffix}">{pathSuffix}</a></td>
+</tr>
+{/FileStatus}
+</tbody>
+</table>

+ 86 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/explorer.html

@@ -0,0 +1,86 @@
+<!--
+    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.
+  -->
+<!DOCTYPE html PUBLIC "-//W3C//DTD XHTML 1.0 Strict//EN"
+	  "http://www.w3.org/TR/xhtml1/DTD/xhtml1-strict.dtd">
+<html xmlns="http://www.w3.org/1999/xhtml">
+  <head>
+    <link rel="stylesheet" type="text/css" href="//netdna.bootstrapcdn.com/bootstrap/3.0.0/css/bootstrap.min.css" />
+    <link rel="stylesheet" type="text/css" href="/static/hadoop.css" />
+    <title>Browsing HDFS</title>
+  </head>
+  <body>
+    <div class="modal" id="file-info" tabindex="-1" role="dialog" aria-hidden="true">
+      <div class="modal-dialog">
+	<div class="modal-content">
+	  <div class="modal-header"><button type="button" class="close" data-dismiss="modal" aria-hidden="true">&times;</button>
+	    <h4 class="modal-title" id="file-info-title">File information</h4>
+	  </div>
+	  <div class="modal-body" id="file-info-body">
+	    <a id="file-info-download">Download</a>
+            <a id="file-info-preview" style="cursor:pointer">Tail the file (last 32K)</a>
+	    <hr />
+	    <div class="panel panel-info" id="file-info-blockinfo-panel">
+	      <div class="panel-heading">
+		Block information -- 
+		<select class="btn btn-default" id="file-info-blockinfo-list">
+		</select>
+	      </div>
+	      <div class="panel-body" id="file-info-blockinfo-body"></div>
+	    </div>
+	    <div class="panel panel-info" id="file-info-tail" style="display:none">
+	      <div class="panel-heading">File contents</div>
+	      <div class="panel-body">
+		<div class="input-group-sm">
+		<textarea class="form-control" style="height: 150px" id="file-info-preview-body"></textarea>
+		</div>
+	      </div>
+	    </div>
+	  </div>
+	  <div class="modal-footer"><button type="button" class="btn btn-primary"
+					    data-dismiss="modal">Close</button></div>
+	</div>
+      </div>
+    </div>
+    <div class="container">
+      <div class="page-header">
+	<h1>Browse Directory</h1>
+      </div>
+      <div class="alert alert-danger" id="alert-panel" style="display:none">
+	<button type="button" class="close" onclick="$('#alert-panel').hide();">&times;</button>
+	<div class="alert-body" id="alert-panel-body"></div>
+      </div>
+      <div class="row">
+	<form onsubmit="return false;">
+	  <div class="input-group"><input type="text" class="form-control" id=
+					  "directory" /> <span class="input-group-btn"><button class="btn btn-default"
+											       type="submit" id="btn-nav-directory"><span class="input-group-btn">Go!</span></button></span></div>
+	</form>
+      </div>
+      <br />
+      <div id="panel"></div>
+    </div>
+    <script type="text/javascript" src="//ajax.googleapis.com/ajax/libs/jquery/2.0.3/jquery.min.js">
+    </script><script type="text/javascript" src="//netdna.bootstrapcdn.com/bootstrap/3.0.0/js/bootstrap.min.js">
+    </script><script type="text/javascript" src="/static/dust-full-2.0.0.min.js">
+    </script><script type="text/javascript" src="/static/dust-helpers-1.1.1.min.js">
+    </script><script type="text/javascript" src="dfs-dust.js">
+    </script><script type="text/javascript" src="explorer.js">
+    </script>
+    <hr />
+    <p><a href="http://hadoop.apache.org/core">Hadoop</a>, 2013.</p>
+  </body>
+</html>

+ 182 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/explorer.js

@@ -0,0 +1,182 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+(function() {
+  "use strict";
+
+  // The chunk size of tailing the files, i.e., how many bytes will be shown
+  // in the preview.
+  var TAIL_CHUNK_SIZE = 32768;
+  var helpers = {
+    'helper_to_permission': function(chunk, ctx, bodies, params) {
+      var p = ctx.current().permission;
+      var dir = ctx.current().type == 'DIRECTORY' ? 'd' : '-';
+      var symbols = [ '---', '--x', '-w-', '-wx', 'r--', 'r-x', 'rw-', 'rwx' ];
+      var sticky = p > 1000;
+
+      var res = "";
+      for (var i = 0; i < 3; ++i) {
+	res = symbols[(p % 10)] + res;
+	p = Math.floor(p / 10);
+      }
+
+      if (sticky) {
+	var exec = ((parms.perm % 10) & 1) == 1;
+	res[res.length - 1] = exec ? 't' : 'T';
+      }
+
+      chunk.write(dir + res);
+      return chunk;
+    }
+  };
+
+  var base = dust.makeBase(helpers);
+  var current_directory = "";
+
+  function show_err_msg(msg) {
+    $('#alert-panel-body').html(msg);
+    $('#alert-panel').show();
+  }
+
+  function network_error_handler(url) {
+    return function (jqxhr, text, err) {
+      var msg = '<p>Failed to retreive data from ' + url + ', cause: ' + err + '</p>';
+      if (url.indexOf('/webhdfs/v1') === 0)  {
+        msg += '<p>WebHDFS might be disabled. WebHDFS is required to browse the filesystem.</p>';
+      }
+      show_err_msg(msg);
+    };
+  }
+
+  function append_path(prefix, s) {
+    var l = prefix.length;
+    var p = l > 0 && prefix[l - 1] == '/' ? prefix.substring(0, l - 1) : prefix;
+    return p + '/' + s;
+  }
+
+  function get_response(data, type) {
+    return data[type] !== undefined ? data[type] : null;
+  }
+
+  function get_response_err_msg(data) {
+    var msg = data.RemoteException !== undefined ? data.RemoteException.message : "";
+    return msg;
+  }
+
+  function view_file_details(path, abs_path) {
+    function show_block_info(blocks) {
+      var menus = $('#file-info-blockinfo-list');
+      menus.empty();
+
+      menus.data("blocks", blocks);
+      menus.change(function() {
+        var d = $(this).data('blocks')[$(this).val()];
+        if (d === undefined) {
+          return;
+        }
+
+        dust.render('block-info', d, function(err, out) {
+          $('#file-info-blockinfo-body').html(out);
+        });
+
+      });
+      for (var i = 0; i < blocks.length; ++i) {
+        var item = $('<option value="' + i + '">Block ' + i + '</option>');
+        menus.append(item);
+      }
+      menus.change();
+    }
+
+    var url = '/webhdfs/v1' + abs_path + '?op=GET_BLOCK_LOCATIONS';
+    $.ajax({"url": url, "crossDomain": true}).done(function(data) {
+      var d = get_response(data, "LocatedBlocks");
+      if (d === null) {
+        show_err_msg(get_response_err_msg(data));
+        return;
+      }
+
+      $('#file-info-tail').hide();
+      $('#file-info-title').text("File information - " + path);
+
+      var download_url = '/webhdfs/v1' + abs_path + '/?op=OPEN';
+
+      $('#file-info-download').attr('href', download_url);
+      $('#file-info-preview').click(function() {
+        var offset = d.fileLength - TAIL_CHUNK_SIZE;
+        var url = offset > 0 ? download_url + '&offset=' + offset : download_url;
+        $.get(url, function(t) {
+          $('#file-info-preview-body').val(t);
+          $('#file-info-tail').show();
+        }, "text").error(network_error_handler(url));
+      });
+
+      if (d.fileLength > 0) {
+        show_block_info(d.locatedBlocks);
+        $('#file-info-blockinfo-panel').show();
+      } else {
+        $('#file-info-blockinfo-panel').hide();
+      }
+      $('#file-info').modal();
+    }).error(network_error_handler(url));
+  }
+
+  function browse_directory(dir) {
+    var url = '/webhdfs/v1' + dir + '?op=LISTSTATUS';
+    $.get(url, function(data) {
+      var d = get_response(data, "FileStatuses");
+      if (d === null) {
+        show_err_msg(get_response_err_msg(data));
+        return;
+      }
+
+      current_directory = dir;
+      $('#directory').val(dir);
+      dust.render('explorer', base.push(d), function(err, out) {
+        $('#panel').html(out);
+
+        $('.explorer-browse-links').click(function() {
+          var type = $(this).attr('inode-type');
+          var path = $(this).attr('inode-path');
+          var abs_path = append_path(current_directory, path);
+          if (type == 'DIRECTORY') {
+            browse_directory(abs_path);
+          } else {
+            view_file_details(path, abs_path);
+          }
+        });
+      });
+    }).error(network_error_handler(url));
+  }
+
+
+  function init() {
+    var templates = [
+      { 'name': 'explorer', 'url': 'explorer.dust.html'},
+      { 'name': 'block-info', 'url': 'explorer-block-info.dust.html'}
+    ];
+
+    load_templates(dust, templates, function () {
+      var b = function() { browse_directory($('#directory').val()); };
+      $('#btn-nav-directory').click(b);
+      browse_directory('/');
+    }, function (url, jqxhr, text, err) {
+      network_error_handler(url)(jqxhr, text, err);
+    });
+  }
+
+  init();
+})();

+ 258 - 0
hadoop-hdfs-project/hadoop-hdfs/src/site/apt/HdfsNfsGateway.apt.vm

@@ -0,0 +1,258 @@
+
+~~ Licensed under the Apache License, Version 2.0 (the "License");
+~~ you may not use this file except in compliance with the License.
+~~ You may obtain a copy of the License at
+~~
+~~   http://www.apache.org/licenses/LICENSE-2.0
+~~
+~~ Unless required by applicable law or agreed to in writing, software
+~~ distributed under the License is distributed on an "AS IS" BASIS,
+~~ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+~~ See the License for the specific language governing permissions and
+~~ limitations under the License. See accompanying LICENSE file.
+
+  ---
+  Hadoop Distributed File System-${project.version} - HDFS NFS Gateway
+  ---
+  ---
+  ${maven.build.timestamp}
+
+HDFS NFS Gateway
+
+  \[ {{{./index.html}Go Back}} \]
+
+%{toc|section=1|fromDepth=0}
+
+* {Overview}
+
+  The NFS Gateway supports NFSv3 and allows HDFS to be mounted as part of the client's local file system.
+  Currently NFS Gateway supports and enables the following usage patterns:
+
+   * Users can browse the HDFS file system through their local file system
+     on NFSv3 client compatible operating systems.
+
+   * Users can download files from the the HDFS file system on to their
+     local file system.
+
+   * Users can upload files from their local file system directly to the
+     HDFS file system.
+
+   * Users can stream data directly to HDFS through the mount point. File
+     append is supported but random write is not supported. 
+
+  The NFS gateway machine needs the same thing to run an HDFS client like Hadoop JAR files, HADOOP_CONF directory.
+  The NFS gateway can be on the same host as DataNode, NameNode, or any HDFS client. 
+
+
+* {Configuration}
+
+   NFS gateway can work with its default settings in most cases. However, it's
+   strongly recommended for the users to update a few configuration properties based on their use
+   cases. All the related configuration properties can be added or updated in hdfs-site.xml.
+  
+   * If the client mounts the export with access time update allowed, make sure the following 
+    property is not disabled in the configuration file. Only NameNode needs to restart after 
+    this property is changed. On some Unix systems, the user can disable access time update
+    by mounting the export with "noatime".
+
+----
+<property>
+  <name>dfs.access.time.precision</name>
+  <value>3600000</value>
+  <description>The access time for HDFS file is precise upto this value. 
+    The default value is 1 hour. Setting a value of 0 disables
+    access times for HDFS.
+  </description>
+</property>
+----
+
+   * Users are expected to update the file dump directory. NFS client often 
+      reorders writes. Sequential writes can arrive at the NFS gateway at random
+      order. This directory is used to temporarily save out-of-order writes
+      before writing to HDFS. For each file, the out-of-order writes are dumped after 
+      they are accumulated to exceed certain threshold (e.g., 1MB) in memory.
+      One needs to make sure the directory has enough
+      space. For example, if the application uploads 10 files with each having 
+      100MB, it is recommended for this directory to have roughly 1GB space in case if a
+      worst-case write reorder happens to every file. Only NFS gateway needs to restart after 
+      this property is updated.
+
+----
+  <property>    
+    <name>dfs.nfs3.dump.dir</name>    
+    <value>/tmp/.hdfs-nfs</value>
+  </property>
+---- 
+
+  * By default, the export can be mounted by any client. To better control the access,
+    users can update the following property. The value string contains machine name and
+    access privilege, separated by whitespace
+    characters. Machine name format can be single host, wildcards, and IPv4 networks.The
+    access privilege uses rw or ro to specify readwrite or readonly access of the machines to exports. If the access
+    privilege is not provided, the default is read-only. Entries are separated by ";".
+    For example: "192.168.0.0/22 rw ; host*.example.com ; host1.test.org ro;". Only NFS gateway needs to restart after 
+    this property is updated.
+
+----
+<property>
+  <name>dfs.nfs.exports.allowed.hosts</name>
+  <value>* rw</value>
+</property>
+----
+
+  * Customize log settings. To get NFS debug trace, users can edit the log4j.property file 
+   to add the following. Note, debug trace, especially for ONCRPC, can be very verbose.
+
+    To change logging level:
+
+----------------------------------------------- 
+    log4j.logger.org.apache.hadoop.hdfs.nfs=DEBUG
+----------------------------------------------- 
+
+    To get more details of ONCRPC requests:
+
+----------------------------------------------- 
+    log4j.logger.org.apache.hadoop.oncrpc=DEBUG
+----------------------------------------------- 
+
+
+* {Start and stop NFS gateway service}
+
+  Three daemons are required to provide NFS service: rpcbind (or portmap), mountd and nfsd.
+  The NFS gateway process has both nfsd and mountd. It shares the HDFS root "/" as the
+  only export. It is recommended to use the portmap included in NFS gateway package. Even
+  though NFS gateway works with portmap/rpcbind provide by most Linux distributions, the
+  package included portmap is needed on some Linux systems such as REHL6.2 due to an 
+  {{{https://bugzilla.redhat.com/show_bug.cgi?id=731542}rpcbind bug}}. More detailed discussions can
+  be found in {{{https://issues.apache.org/jira/browse/HDFS-4763}HDFS-4763}}.
+
+   [[1]] Stop nfs/rpcbind/portmap services provided by the platform (commands can be different on various Unix platforms):
+      
+-------------------------
+     service nfs stop
+      
+     service rpcbind stop
+-------------------------
+
+
+   [[2]] Start package included portmap (needs root privileges):
+
+-------------------------
+     hadoop portmap
+  
+     OR
+
+     hadoop-daemon.sh start portmap
+-------------------------
+
+   [[3]] Start mountd and nfsd.
+   
+     No root privileges are required for this command. However, ensure that the user starting
+     the Hadoop cluster and the user starting the NFS gateway are same.
+
+-------------------------
+     hadoop nfs3
+
+     OR
+
+     hadoop-daemon.sh start nfs3
+-------------------------
+
+     Note, if the hadoop-daemon.sh script starts the NFS gateway, its log can be found in the hadoop log folder.
+
+
+   [[4]] Stop NFS gateway services.
+
+-------------------------
+      hadoop-daemon.sh stop nfs3
+
+      hadoop-daemon.sh stop portmap
+-------------------------
+
+
+* {Verify validity of NFS related services}
+
+    [[1]] Execute the following command to verify if all the services are up and running:
+
+-------------------------
+       rpcinfo -p $nfs_server_ip
+-------------------------
+
+     You should see output similar to the following:
+
+-------------------------
+       program vers proto   port
+
+       100005    1   tcp   4242  mountd
+
+       100005    2   udp   4242  mountd
+
+       100005    2   tcp   4242  mountd
+
+       100000    2   tcp    111  portmapper
+
+       100000    2   udp    111  portmapper
+
+       100005    3   udp   4242  mountd
+
+       100005    1   udp   4242  mountd
+
+       100003    3   tcp   2049  nfs
+
+       100005    3   tcp   4242  mountd
+-------------------------
+
+    [[2]]  Verify if the HDFS namespace is exported and can be mounted.
+
+-------------------------
+        showmount -e $nfs_server_ip                         
+-------------------------
+
+      You should see output similar to the following:
+     
+-------------------------
+        Exports list on $nfs_server_ip :
+
+        / (everyone)
+-------------------------
+
+
+* {Mount the export “/”}
+
+  Currently NFS v3 only uses TCP as the transportation protocol. 
+  NLM is not supported so mount option "nolock" is needed. It's recommended to use
+  hard mount. This is because, even after the client sends all data to 
+  NFS gateway, it may take NFS gateway some extra time to transfer data to HDFS 
+  when writes were reorderd by NFS client Kernel.
+ 
+  If soft mount has to be used, the user should give it a relatively 
+  long timeout (at least no less than the default timeout on the host) .
+
+  The users can mount the HDFS namespace as shown below:
+
+-------------------------------------------------------------------  
+       mount -t nfs -o vers=3,proto=tcp,nolock $server:/  $mount_point
+-------------------------------------------------------------------
+
+  Then the users can access HDFS as part of the local file system except that, 
+  hard link and random write are not supported yet.
+
+* {User authentication and mapping}
+
+  NFS gateway in this release uses AUTH_UNIX style authentication. When the user on NFS client
+  accesses the mount point, NFS client passes the UID to NFS gateway. 
+  NFS gateway does a lookup to find user name from the UID, and then passes the
+  username to the HDFS along with the HDFS requests.
+  For example, if the NFS client has current user as "admin", when the user accesses
+  the mounted directory, NFS gateway will access HDFS as user "admin". To access HDFS
+  as the user "hdfs", one needs to switch the current user to "hdfs" on the client system
+  when accessing the mounted directory.
+
+  The system administrator must ensure that the user on NFS client host has the same
+  name and UID as that on the NFS gateway host. This is usually not a problem if
+  the same user management system (e.g., LDAP/NIS) is used to create and deploy users on
+  HDFS nodes and NFS client node. In case the user account is created manually in different hosts, one might need to 
+  modify UID (e.g., do "usermod -u 123 myusername") on either NFS client or NFS gateway host
+  in order to make it the same on both sides. More technical details of RPC AUTH_UNIX can be found
+  in {{{http://tools.ietf.org/html/rfc1057}RPC specification}}.
+

+ 74 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFsck.java

@@ -83,7 +83,6 @@ import org.apache.log4j.RollingFileAppender;
 import org.junit.Test;
 
 import com.google.common.collect.Sets;
-import org.mockito.Mockito;
 import static org.mockito.Mockito.*;
 
 /**
@@ -892,6 +891,80 @@ public class TestFsck {
       }
     }
   }
+  
+  /**
+   * Tests that the # of misreplaced replicas is correct
+   * @throws IOException
+   */
+  @Test
+  public void testFsckMisPlacedReplicas() throws IOException {
+    // Desired replication factor
+    final short REPL_FACTOR = 2;
+    // Number of replicas to actually start
+    short NUM_DN = 2;
+    // Number of blocks to write
+    final short NUM_BLOCKS = 3;
+    // Set a small-ish blocksize
+    final long blockSize = 512;
+    
+    String [] racks = {"/rack1", "/rack1"};
+    String [] hosts = {"host1", "host2"};
+    
+    Configuration conf = new Configuration();
+    conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, blockSize);
+    
+    MiniDFSCluster cluster = null;
+    DistributedFileSystem dfs = null;
+    
+    try {
+      // Startup a minicluster
+      cluster = 
+          new MiniDFSCluster.Builder(conf).numDataNodes(NUM_DN).hosts(hosts)
+          .racks(racks).build();
+      assertNotNull("Failed Cluster Creation", cluster);
+      cluster.waitClusterUp();
+      dfs = (DistributedFileSystem) cluster.getFileSystem();
+      assertNotNull("Failed to get FileSystem", dfs);
+      
+      // Create a file that will be intentionally under-replicated
+      final String pathString = new String("/testfile");
+      final Path path = new Path(pathString);
+      long fileLen = blockSize * NUM_BLOCKS;
+      DFSTestUtil.createFile(dfs, path, fileLen, REPL_FACTOR, 1);
+      
+      // Create an under-replicated file
+      NameNode namenode = cluster.getNameNode();
+      NetworkTopology nettop = cluster.getNamesystem().getBlockManager()
+          .getDatanodeManager().getNetworkTopology();
+      // Add a new node on different rack, so previous blocks' replicas 
+      // are considered to be misplaced
+      nettop.add(DFSTestUtil.getDatanodeDescriptor("/rack2", "/host3"));
+      NUM_DN++;
+      
+      Map<String,String[]> pmap = new HashMap<String, String[]>();
+      Writer result = new StringWriter();
+      PrintWriter out = new PrintWriter(result, true);
+      InetAddress remoteAddress = InetAddress.getLocalHost();
+      NamenodeFsck fsck = new NamenodeFsck(conf, namenode, nettop, pmap, out, 
+          NUM_DN, (short)REPL_FACTOR, remoteAddress);
+      
+      // Run the fsck and check the Result
+      final HdfsFileStatus file = 
+          namenode.getRpcServer().getFileInfo(pathString);
+      assertNotNull(file);
+      Result res = new Result(conf);
+      fsck.check(pathString, file, res);
+      // check misReplicatedBlock number.
+      assertEquals(res.numMisReplicatedBlocks, NUM_BLOCKS);
+    } finally {
+      if(dfs != null) {
+        dfs.close();
+      }
+      if(cluster != null) {
+        cluster.shutdown();
+      }
+    }
+  }
 
   /** Test fsck with FileNotFound */
   @Test

+ 1 - 0
hadoop-project/src/site/site.xml

@@ -80,6 +80,7 @@
       <item name="HttpFS Gateway" href="hadoop-hdfs-httpfs/index.html"/>
       <item name="Short Circuit Local Reads" 
           href="hadoop-project-dist/hadoop-hdfs/ShortCircuitLocalReads.html"/>
+      <item name="HDFS NFS Gateway" href="hadoop-project-dist/hadoop-hdfs/HdfsNfsGateway.html"/>
     </menu>
 
     <menu name="MapReduce" inherit="top">

+ 9 - 2
hadoop-yarn-project/CHANGES.txt

@@ -61,8 +61,6 @@ Release 2.3.0 - UNRELEASED
 
     YARN-976. Document the meaning of a virtual core. (Sandy Ryza)
 
-    YARN-1258. Allow configuring the Fair Scheduler root queue (Sandy Ryza)
-
     YARN-1182. MiniYARNCluster creates and inits the RM/NM only on start()
     (Karthik Kambatla via Sandy Ryza)
 
@@ -95,6 +93,12 @@ Release 2.2.1 - UNRELEASED
     YARN-305. Fair scheduler logs too many "Node offered to app" messages.
     (Lohit Vijayarenu via Sandy Ryza)
 
+    YARN-1258. Allow configuring the Fair Scheduler root queue (Sandy Ryza)
+
+    YARN-1288. Make Fair Scheduler ACLs more user friendly (Sandy Ryza)
+
+    YARN-1315. TestQueueACLs should also test FairScheduler (Sandy Ryza)
+
   OPTIMIZATIONS
 
   BUG FIXES
@@ -123,6 +127,9 @@ Release 2.2.1 - UNRELEASED
     YARN-1185. Fixed FileSystemRMStateStore to not leave partial files that
     prevent subsequent ResourceManager recovery. (Omkar Vinit Joshi via vinodkv)
 
+    YARN-1331. yarn.cmd exits with NoClassDefFoundError trying to run rmadmin or
+    logs. (cnauroth)
+
 Release 2.2.0 - 2013-10-13
 
   INCOMPATIBLE CHANGES

+ 2 - 3
hadoop-yarn-project/hadoop-yarn/bin/yarn.cmd

@@ -149,7 +149,7 @@ goto :eof
   goto :eof
 
 :rmadmin
-  set CLASS=org.apache.hadoop.yarn.server.resourcemanager.tools.RMAdmin
+  set CLASS=org.apache.hadoop.yarn.client.cli.RMAdminCLI
   set YARN_OPTS=%YARN_OPTS% %YARN_CLIENT_OPTS%
   goto :eof
 
@@ -200,7 +200,7 @@ goto :eof
   goto :eof
 
 :logs
-  set CLASS=org.apache.hadoop.yarn.logaggregation.LogDumper
+  set CLASS=org.apache.hadoop.yarn.client.cli.LogsCLI
   set YARN_OPTS=%YARN_OPTS% %YARN_CLIENT_OPTS%
   goto :eof
 
@@ -237,7 +237,6 @@ goto :eof
   @echo        where COMMAND is one of:
   @echo   resourcemanager      run the ResourceManager
   @echo   nodemanager          run a nodemanager on each slave
-  @echo   historyserver        run job history servers as a standalone daemon
   @echo   rmadmin              admin tools
   @echo   version              print the version
   @echo   jar ^<jar^>          run a jar file

+ 0 - 8
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/Queue.java

@@ -19,12 +19,10 @@
 package org.apache.hadoop.yarn.server.resourcemanager.scheduler;
 
 import java.util.List;
-import java.util.Map;
 
 import org.apache.hadoop.classification.InterfaceAudience.LimitedPrivate;
 import org.apache.hadoop.classification.InterfaceStability.Evolving;
 import org.apache.hadoop.security.UserGroupInformation;
-import org.apache.hadoop.security.authorize.AccessControlList;
 import org.apache.hadoop.yarn.api.records.QueueACL;
 import org.apache.hadoop.yarn.api.records.QueueInfo;
 import org.apache.hadoop.yarn.api.records.QueueUserACLInfo;
@@ -44,12 +42,6 @@ public interface Queue {
    */
   QueueMetrics getMetrics();
 
-  /**
-   * Get ACLs for the queue.
-   * @return ACLs for the queue
-   */
-  public Map<QueueACL, AccessControlList> getQueueAcls();
-  
   /**
    * Get queue information
    * @param includeChildQueues include child queues?

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

@@ -526,11 +526,6 @@ public class LeafQueue implements CSQueue {
     return userLimitFactor;
   }
 
-  @Override
-  public synchronized Map<QueueACL, AccessControlList> getQueueAcls() {
-    return new HashMap<QueueACL, AccessControlList>(acls);
-  }
-
   @Override
   public synchronized QueueInfo getQueueInfo(
       boolean includeChildQueues, boolean recursive) {

+ 0 - 5
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/ParentQueue.java

@@ -299,11 +299,6 @@ public class ParentQueue implements CSQueue {
     return state;
   }
 
-  @Override
-  public synchronized Map<QueueACL, AccessControlList> getQueueAcls() {
-    return new HashMap<QueueACL, AccessControlList>(acls);
-  }
-
   @Override
   public synchronized QueueInfo getQueueInfo( 
       boolean includeChildQueues, boolean recursive) {

+ 1 - 4
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSLeafQueue.java

@@ -24,14 +24,12 @@ import java.util.Collections;
 import java.util.Comparator;
 import java.util.Iterator;
 import java.util.List;
-import java.util.Map;
 
 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.security.UserGroupInformation;
-import org.apache.hadoop.security.authorize.AccessControlList;
 import org.apache.hadoop.yarn.api.records.QueueACL;
 import org.apache.hadoop.yarn.api.records.QueueUserACLInfo;
 import org.apache.hadoop.yarn.api.records.Resource;
@@ -177,8 +175,7 @@ public class FSLeafQueue extends FSQueue {
       recordFactory.newRecordInstance(QueueUserACLInfo.class);
     List<QueueACL> operations = new ArrayList<QueueACL>();
     for (QueueACL operation : QueueACL.values()) {
-      Map<QueueACL, AccessControlList> acls = queueMgr.getQueueAcls(getName());
-      if (acls.get(operation).isUserAllowed(user)) {
+      if (hasAccess(operation, user)) {
         operations.add(operation);
       }
     }

+ 1 - 10
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSQueue.java

@@ -20,13 +20,10 @@ package org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair;
 
 import java.util.ArrayList;
 import java.util.Collection;
-import java.util.HashMap;
-import java.util.Map;
 
 import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.security.UserGroupInformation;
-import org.apache.hadoop.security.authorize.AccessControlList;
 import org.apache.hadoop.yarn.api.records.Priority;
 import org.apache.hadoop.yarn.api.records.QueueACL;
 import org.apache.hadoop.yarn.api.records.QueueInfo;
@@ -135,12 +132,6 @@ public abstract class FSQueue extends Schedulable implements Queue {
     return queueInfo;
   }
   
-  @Override
-  public Map<QueueACL, AccessControlList> getQueueAcls() {
-    Map<QueueACL, AccessControlList> acls = queueMgr.getQueueAcls(getName());
-    return new HashMap<QueueACL, AccessControlList>(acls);
-  }
-  
   @Override
   public FSQueueMetrics getMetrics() {
     return metrics;
@@ -154,7 +145,7 @@ public abstract class FSQueue extends Schedulable implements Queue {
   
   public boolean hasAccess(QueueACL acl, UserGroupInformation user) {
     // Check if the leaf-queue allows access
-    if (queueMgr.getQueueAcls(getName()).get(acl).isUserAllowed(user)) {
+    if (queueMgr.getQueueAcl(getName(), acl).isUserAllowed(user)) {
       return true;
     }
 

+ 5 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FairSchedulerConfiguration.java

@@ -50,7 +50,7 @@ public class FairSchedulerConfiguration extends Configuration {
   
   private static final String CONF_PREFIX =  "yarn.scheduler.fair.";
 
-  protected static final String ALLOCATION_FILE = CONF_PREFIX + "allocation.file";
+  public static final String ALLOCATION_FILE = CONF_PREFIX + "allocation.file";
   protected static final String DEFAULT_ALLOCATION_FILE = "fair-scheduler.xml";
   protected static final String EVENT_LOG_DIR = "eventlog.dir";
 
@@ -113,6 +113,10 @@ public class FairSchedulerConfiguration extends Configuration {
   protected static final String MAX_ASSIGN = CONF_PREFIX + "max.assign";
   protected static final int DEFAULT_MAX_ASSIGN = -1;
 
+  public FairSchedulerConfiguration() {
+    super();
+  }
+  
   public FairSchedulerConfiguration(Configuration conf) {
     super(conf);
   }

+ 13 - 24
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/QueueManager.java

@@ -72,6 +72,9 @@ public class QueueManager {
    * (this is done to prevent loading a file that hasn't been fully written).
    */
   public static final long ALLOC_RELOAD_WAIT = 5 * 1000;
+  
+  private static final AccessControlList EVERYBODY_ACL = new AccessControlList("*");
+  private static final AccessControlList NOBODY_ACL = new AccessControlList(" ");
 
   private final FairScheduler scheduler;
 
@@ -381,15 +384,6 @@ public class QueueManager {
         queueMetrics.setMinShare(queue.getMinShare());
         queueMetrics.setMaxShare(queue.getMaxShare());
       }
-      
-      // Root queue should have empty ACLs.  As a queue's ACL is the union of
-      // its ACL and all its parents' ACLs, setting the roots' to empty will
-      // neither allow nor prohibit more access to its children.
-      Map<QueueACL, AccessControlList> rootAcls =
-          new HashMap<QueueACL, AccessControlList>();
-      rootAcls.put(QueueACL.SUBMIT_APPLICATIONS, new AccessControlList(" "));
-      rootAcls.put(QueueACL.ADMINISTER_QUEUE, new AccessControlList(" "));
-      queueAcls.put(ROOT_QUEUE, rootAcls);
  
       // Create all queus
       for (String name: queueNamesInAllocFile) {
@@ -454,10 +448,10 @@ public class QueueManager {
         policy.initialize(scheduler.getClusterCapacity());
         queuePolicies.put(queueName, policy);
       } else if ("aclSubmitApps".equals(field.getTagName())) {
-        String text = ((Text)field.getFirstChild()).getData().trim();
+        String text = ((Text)field.getFirstChild()).getData();
         acls.put(QueueACL.SUBMIT_APPLICATIONS, new AccessControlList(text));
       } else if ("aclAdministerApps".equals(field.getTagName())) {
-        String text = ((Text)field.getFirstChild()).getData().trim();
+        String text = ((Text)field.getFirstChild()).getData();
         acls.put(QueueACL.ADMINISTER_QUEUE, new AccessControlList(text));
       } else if ("queue".endsWith(field.getTagName()) || 
           "pool".equals(field.getTagName())) {
@@ -577,21 +571,16 @@ public class QueueManager {
 
   /**
    * Get the ACLs associated with this queue. If a given ACL is not explicitly
-   * configured, include the default value for that ACL.
+   * configured, include the default value for that ACL.  The default for the
+   * root queue is everybody ("*") and the default for all other queues is
+   * nobody ("")
    */
-  public Map<QueueACL, AccessControlList> getQueueAcls(String queue) {
-    HashMap<QueueACL, AccessControlList> out = new HashMap<QueueACL, AccessControlList>();
-    Map<QueueACL, AccessControlList> queueAcl = info.queueAcls.get(queue);
-    if (queueAcl != null) {
-      out.putAll(queueAcl);
-    }
-    if (!out.containsKey(QueueACL.ADMINISTER_QUEUE)) {
-      out.put(QueueACL.ADMINISTER_QUEUE, new AccessControlList("*"));
-    }
-    if (!out.containsKey(QueueACL.SUBMIT_APPLICATIONS)) {
-      out.put(QueueACL.SUBMIT_APPLICATIONS, new AccessControlList("*"));
+  public AccessControlList getQueueAcl(String queue, QueueACL operation) {
+    Map<QueueACL, AccessControlList> queueAcls = info.queueAcls.get(queue);
+    if (queueAcls == null || !queueAcls.containsKey(operation)) {
+      return (queue.equals(ROOT_QUEUE)) ? EVERYBODY_ACL : NOBODY_ACL;
     }
-    return out;
+    return queueAcls.get(operation);
   }
   
   static class QueueManagerInfo {

+ 0 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fifo/FifoScheduler.java

@@ -156,7 +156,6 @@ public class FifoScheduler implements ResourceScheduler, Configurable {
       return queueInfo;
     }
 
-    @Override
     public Map<QueueACL, AccessControlList> getQueueAcls() {
       Map<QueueACL, AccessControlList> acls =
         new HashMap<QueueACL, AccessControlList>();

+ 27 - 65
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestQueueACLs.java → hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/QueueACLsTestBase.java

@@ -42,41 +42,43 @@ import org.apache.hadoop.yarn.api.records.ApplicationAccessType;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
 import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
-import org.apache.hadoop.yarn.api.records.QueueACL;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.ipc.YarnRPC;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppState;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration;
 import org.apache.hadoop.yarn.server.utils.BuilderUtils;
-import org.junit.AfterClass;
-import org.junit.BeforeClass;
+import org.junit.After;
+import org.junit.Before;
 import org.junit.Test;
 
-public class TestQueueACLs {
+public abstract class QueueACLsTestBase {
 
-  private static final String COMMON_USER = "common_user";
-  private static final String QUEUE_A_USER = "queueA_user";
-  private static final String QUEUE_B_USER = "queueB_user";
-  private static final String ROOT_ADMIN = "root_admin";
-  private static final String QUEUE_A_ADMIN = "queueA_admin";
-  private static final String QUEUE_B_ADMIN = "queueB_admin";
+  protected static final String COMMON_USER = "common_user";
+  protected static final String QUEUE_A_USER = "queueA_user";
+  protected static final String QUEUE_B_USER = "queueB_user";
+  protected static final String ROOT_ADMIN = "root_admin";
+  protected static final String QUEUE_A_ADMIN = "queueA_admin";
+  protected static final String QUEUE_B_ADMIN = "queueB_admin";
 
-  private static final String QUEUEA = "queueA";
-  private static final String QUEUEB = "queueB";
+  protected static final String QUEUEA = "queueA";
+  protected static final String QUEUEB = "queueB";
 
   private static final Log LOG = LogFactory.getLog(TestApplicationACLs.class);
 
-  static MockRM resourceManager;
-  static Configuration conf = createConfiguration();
-  final static YarnRPC rpc = YarnRPC.create(conf);
-  final static InetSocketAddress rmAddress = conf.getSocketAddr(
-    YarnConfiguration.RM_ADDRESS, YarnConfiguration.DEFAULT_RM_ADDRESS,
-    YarnConfiguration.DEFAULT_RM_PORT);
-
-  @BeforeClass
-  public static void setup() throws InterruptedException, IOException {
+  MockRM resourceManager;
+  Configuration conf;
+  YarnRPC rpc;
+  InetSocketAddress rmAddress;
+
+  @Before
+  public void setup() throws InterruptedException, IOException {
+    conf = createConfiguration();
+    rpc = YarnRPC.create(conf);
+    rmAddress = conf.getSocketAddr(
+      YarnConfiguration.RM_ADDRESS, YarnConfiguration.DEFAULT_RM_ADDRESS,
+      YarnConfiguration.DEFAULT_RM_PORT);
+    
     AccessControlList adminACL = new AccessControlList("");
     conf.set(YarnConfiguration.YARN_ADMIN_ACL, adminACL.getAclString());
 
@@ -109,8 +111,8 @@ public class TestQueueACLs {
     }
   }
 
-  @AfterClass
-  public static void tearDown() {
+  @After
+  public void tearDown() {
     if (resourceManager != null) {
       resourceManager.stop();
     }
@@ -262,45 +264,5 @@ public class TestQueueACLs {
     return userClient;
   }
 
-  private static YarnConfiguration createConfiguration() {
-    CapacitySchedulerConfiguration csConf =
-        new CapacitySchedulerConfiguration();
-    csConf.setQueues(CapacitySchedulerConfiguration.ROOT, new String[] {
-        QUEUEA, QUEUEB });
-
-    csConf.setCapacity(CapacitySchedulerConfiguration.ROOT + "." + QUEUEA, 50f);
-    csConf.setCapacity(CapacitySchedulerConfiguration.ROOT + "." + QUEUEB, 50f);
-
-    Map<QueueACL, AccessControlList> aclsOnQueueA =
-        new HashMap<QueueACL, AccessControlList>();
-    AccessControlList submitACLonQueueA = new AccessControlList(QUEUE_A_USER);
-    submitACLonQueueA.addUser(COMMON_USER);
-    AccessControlList adminACLonQueueA = new AccessControlList(QUEUE_A_ADMIN);
-    aclsOnQueueA.put(QueueACL.SUBMIT_APPLICATIONS, submitACLonQueueA);
-    aclsOnQueueA.put(QueueACL.ADMINISTER_QUEUE, adminACLonQueueA);
-    csConf.setAcls(CapacitySchedulerConfiguration.ROOT + "." + QUEUEA,
-      aclsOnQueueA);
-
-    Map<QueueACL, AccessControlList> aclsOnQueueB =
-        new HashMap<QueueACL, AccessControlList>();
-    AccessControlList submitACLonQueueB = new AccessControlList(QUEUE_B_USER);
-    submitACLonQueueB.addUser(COMMON_USER);
-    AccessControlList adminACLonQueueB = new AccessControlList(QUEUE_B_ADMIN);
-    aclsOnQueueB.put(QueueACL.SUBMIT_APPLICATIONS, submitACLonQueueB);
-    aclsOnQueueB.put(QueueACL.ADMINISTER_QUEUE, adminACLonQueueB);
-    csConf.setAcls(CapacitySchedulerConfiguration.ROOT + "." + QUEUEB,
-      aclsOnQueueB);
-
-    Map<QueueACL, AccessControlList> aclsOnRootQueue =
-        new HashMap<QueueACL, AccessControlList>();
-    AccessControlList submitACLonRoot = new AccessControlList("");
-    AccessControlList adminACLonRoot = new AccessControlList(ROOT_ADMIN);
-    aclsOnRootQueue.put(QueueACL.SUBMIT_APPLICATIONS, submitACLonRoot);
-    aclsOnRootQueue.put(QueueACL.ADMINISTER_QUEUE, adminACLonRoot);
-    csConf.setAcls(CapacitySchedulerConfiguration.ROOT, aclsOnRootQueue);
-
-    YarnConfiguration conf = new YarnConfiguration(csConf);
-    conf.setBoolean(YarnConfiguration.YARN_ACL_ENABLE, true);
-    return conf;
-  }
+  protected abstract Configuration createConfiguration() throws IOException;
 }

+ 73 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacitySchedulerQueueACLs.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.resourcemanager.scheduler.capacity;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.security.authorize.AccessControlList;
+import org.apache.hadoop.yarn.api.records.QueueACL;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.server.resourcemanager.QueueACLsTestBase;
+
+public class TestCapacitySchedulerQueueACLs extends QueueACLsTestBase {
+  @Override
+  protected Configuration createConfiguration() {
+    CapacitySchedulerConfiguration csConf =
+        new CapacitySchedulerConfiguration();
+    csConf.setQueues(CapacitySchedulerConfiguration.ROOT, new String[] {
+        QUEUEA, QUEUEB });
+
+    csConf.setCapacity(CapacitySchedulerConfiguration.ROOT + "." + QUEUEA, 50f);
+    csConf.setCapacity(CapacitySchedulerConfiguration.ROOT + "." + QUEUEB, 50f);
+
+    Map<QueueACL, AccessControlList> aclsOnQueueA =
+        new HashMap<QueueACL, AccessControlList>();
+    AccessControlList submitACLonQueueA = new AccessControlList(QUEUE_A_USER);
+    submitACLonQueueA.addUser(COMMON_USER);
+    AccessControlList adminACLonQueueA = new AccessControlList(QUEUE_A_ADMIN);
+    aclsOnQueueA.put(QueueACL.SUBMIT_APPLICATIONS, submitACLonQueueA);
+    aclsOnQueueA.put(QueueACL.ADMINISTER_QUEUE, adminACLonQueueA);
+    csConf.setAcls(CapacitySchedulerConfiguration.ROOT + "." + QUEUEA,
+      aclsOnQueueA);
+
+    Map<QueueACL, AccessControlList> aclsOnQueueB =
+        new HashMap<QueueACL, AccessControlList>();
+    AccessControlList submitACLonQueueB = new AccessControlList(QUEUE_B_USER);
+    submitACLonQueueB.addUser(COMMON_USER);
+    AccessControlList adminACLonQueueB = new AccessControlList(QUEUE_B_ADMIN);
+    aclsOnQueueB.put(QueueACL.SUBMIT_APPLICATIONS, submitACLonQueueB);
+    aclsOnQueueB.put(QueueACL.ADMINISTER_QUEUE, adminACLonQueueB);
+    csConf.setAcls(CapacitySchedulerConfiguration.ROOT + "." + QUEUEB,
+      aclsOnQueueB);
+
+    Map<QueueACL, AccessControlList> aclsOnRootQueue =
+        new HashMap<QueueACL, AccessControlList>();
+    AccessControlList submitACLonRoot = new AccessControlList("");
+    AccessControlList adminACLonRoot = new AccessControlList(ROOT_ADMIN);
+    aclsOnRootQueue.put(QueueACL.SUBMIT_APPLICATIONS, submitACLonRoot);
+    aclsOnRootQueue.put(QueueACL.ADMINISTER_QUEUE, adminACLonRoot);
+    csConf.setAcls(CapacitySchedulerConfiguration.ROOT, aclsOnRootQueue);
+
+    csConf.setBoolean(YarnConfiguration.YARN_ACL_ENABLE, true);
+    csConf.set("yarn.resourcemanager.scheduler.class", CapacityScheduler.class.getName());
+
+    return csConf;
+  }
+}

+ 39 - 30
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFairScheduler.java

@@ -865,22 +865,25 @@ public class TestFairScheduler {
     assertEquals(10, queueManager.getUserMaxApps("user1"));
     assertEquals(5, queueManager.getUserMaxApps("user2"));
 
+    // Root should get * ACL
+    assertEquals("*",queueManager.getQueueAcl("root",
+        QueueACL.ADMINISTER_QUEUE).getAclString());
+    assertEquals("*", queueManager.getQueueAcl("root",
+        QueueACL.SUBMIT_APPLICATIONS).getAclString());
+
     // Unspecified queues should get default ACL
-    Map<QueueACL, AccessControlList> aclsA = queueManager.getQueueAcls("root.queueA");
-    assertTrue(aclsA.containsKey(QueueACL.ADMINISTER_QUEUE));
-    assertEquals("*", aclsA.get(QueueACL.ADMINISTER_QUEUE).getAclString());
-    assertTrue(aclsA.containsKey(QueueACL.SUBMIT_APPLICATIONS));
-    assertEquals("*", aclsA.get(QueueACL.SUBMIT_APPLICATIONS).getAclString());
+    assertEquals(" ",queueManager.getQueueAcl("root.queueA",
+        QueueACL.ADMINISTER_QUEUE).getAclString());
+    assertEquals(" ", queueManager.getQueueAcl("root.queueA",
+        QueueACL.SUBMIT_APPLICATIONS).getAclString());
 
     // Queue B ACL
-    Map<QueueACL, AccessControlList> aclsB = queueManager.getQueueAcls("root.queueB");
-    assertTrue(aclsB.containsKey(QueueACL.ADMINISTER_QUEUE));
-    assertEquals("alice,bob admins", aclsB.get(QueueACL.ADMINISTER_QUEUE).getAclString());
+    assertEquals("alice,bob admins",queueManager.getQueueAcl("root.queueB",
+        QueueACL.ADMINISTER_QUEUE).getAclString());
 
-    // Queue c ACL
-    Map<QueueACL, AccessControlList> aclsC = queueManager.getQueueAcls("root.queueC");
-    assertTrue(aclsC.containsKey(QueueACL.SUBMIT_APPLICATIONS));
-    assertEquals("alice,bob admins", aclsC.get(QueueACL.SUBMIT_APPLICATIONS).getAclString());
+    // Queue C ACL
+    assertEquals("alice,bob admins",queueManager.getQueueAcl("root.queueC",
+        QueueACL.SUBMIT_APPLICATIONS).getAclString());
 
     assertEquals(120000, queueManager.getMinSharePreemptionTimeout("root." + 
         YarnConfiguration.DEFAULT_QUEUE_NAME));
@@ -1063,21 +1066,19 @@ public class TestFairScheduler {
     assertEquals(5, queueManager.getUserMaxApps("user2"));
 
     // Unspecified queues should get default ACL
-    Map<QueueACL, AccessControlList> aclsA = queueManager.getQueueAcls("queueA");
-    assertTrue(aclsA.containsKey(QueueACL.ADMINISTER_QUEUE));
-    assertEquals("*", aclsA.get(QueueACL.ADMINISTER_QUEUE).getAclString());
-    assertTrue(aclsA.containsKey(QueueACL.SUBMIT_APPLICATIONS));
-    assertEquals("*", aclsA.get(QueueACL.SUBMIT_APPLICATIONS).getAclString());
+    assertEquals(" ", queueManager.getQueueAcl("root.queueA",
+        QueueACL.ADMINISTER_QUEUE).getAclString());
+    assertEquals(" ", queueManager.getQueueAcl("root.queueA",
+        QueueACL.SUBMIT_APPLICATIONS).getAclString());
 
     // Queue B ACL
-    Map<QueueACL, AccessControlList> aclsB = queueManager.getQueueAcls("root.queueB");
-    assertTrue(aclsB.containsKey(QueueACL.ADMINISTER_QUEUE));
-    assertEquals("alice,bob admins", aclsB.get(QueueACL.ADMINISTER_QUEUE).getAclString());
+    assertEquals("alice,bob admins", queueManager.getQueueAcl("root.queueB",
+        QueueACL.ADMINISTER_QUEUE).getAclString());
+
+    // Queue C ACL
+    assertEquals("alice,bob admins", queueManager.getQueueAcl("root.queueC",
+        QueueACL.SUBMIT_APPLICATIONS).getAclString());
 
-    // Queue c ACL
-    Map<QueueACL, AccessControlList> aclsC = queueManager.getQueueAcls("root.queueC");
-    assertTrue(aclsC.containsKey(QueueACL.SUBMIT_APPLICATIONS));
-    assertEquals("alice,bob admins", aclsC.get(QueueACL.SUBMIT_APPLICATIONS).getAclString());
 
     assertEquals(120000, queueManager.getMinSharePreemptionTimeout("root." +
         YarnConfiguration.DEFAULT_QUEUE_NAME));
@@ -1664,9 +1665,13 @@ public class TestFairScheduler {
     PrintWriter out = new PrintWriter(new FileWriter(ALLOC_FILE));
     out.println("<?xml version=\"1.0\"?>");
     out.println("<allocations>");
-    out.println("<queue name=\"queue1\">");
-    out.println("<aclSubmitApps>norealuserhasthisname</aclSubmitApps>");
-    out.println("<aclAdministerApps>norealuserhasthisname</aclAdministerApps>");
+    out.println("<queue name=\"root\">");
+    out.println("  <aclSubmitApps> </aclSubmitApps>");
+    out.println("  <aclAdministerApps> </aclAdministerApps>");
+    out.println("  <queue name=\"queue1\">");
+    out.println("    <aclSubmitApps>norealuserhasthisname</aclSubmitApps>");
+    out.println("    <aclAdministerApps>norealuserhasthisname</aclAdministerApps>");
+    out.println("  </queue>");
     out.println("</queue>");
     out.println("</allocations>");
     out.close();
@@ -1893,9 +1898,13 @@ public class TestFairScheduler {
     PrintWriter out = new PrintWriter(new FileWriter(ALLOC_FILE));
     out.println("<?xml version=\"1.0\"?>");
     out.println("<allocations>");
-    out.println("<queue name=\"queue1\">");
-    out.println("<aclSubmitApps>userallow</aclSubmitApps>");
-    out.println("<aclAdministerApps>userallow</aclAdministerApps>");
+    out.println("<queue name=\"root\">");
+    out.println("  <aclSubmitApps> </aclSubmitApps>");
+    out.println("  <aclAdministerApps> </aclAdministerApps>");
+    out.println("  <queue name=\"queue1\">");
+    out.println("    <aclSubmitApps>userallow</aclSubmitApps>");
+    out.println("    <aclAdministerApps>userallow</aclAdministerApps>");
+    out.println("  </queue>");
     out.println("</queue>");
     out.println("</allocations>");
     out.close();

+ 62 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFairSchedulerQueueACLs.java

@@ -0,0 +1,62 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair;
+
+import java.io.File;
+import java.io.FileWriter;
+import java.io.IOException;
+import java.io.PrintWriter;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.server.resourcemanager.QueueACLsTestBase;
+
+public class TestFairSchedulerQueueACLs extends QueueACLsTestBase {
+  @Override
+  protected Configuration createConfiguration() throws IOException {
+    FairSchedulerConfiguration fsConf = new FairSchedulerConfiguration();
+    
+    final String TEST_DIR = new File(System.getProperty("test.build.data",
+        "/tmp")).getAbsolutePath();
+    final String ALLOC_FILE = new File(TEST_DIR, "test-queues.xml")
+        .getAbsolutePath();
+    PrintWriter out = new PrintWriter(new FileWriter(ALLOC_FILE));
+    out.println("<?xml version=\"1.0\"?>");
+    out.println("<allocations>");
+    out.println("<queue name=\"root\">");
+    out.println("  <aclSubmitApps> </aclSubmitApps>");
+    out.println("  <aclAdministerApps>root_admin </aclAdministerApps>");
+    out.println("  <queue name=\"queueA\">");
+    out.println("    <aclSubmitApps>queueA_user,common_user </aclSubmitApps>");
+    out.println("    <aclAdministerApps>queueA_admin </aclAdministerApps>");
+    out.println("  </queue>");
+    out.println("  <queue name=\"queueB\">");
+    out.println("    <aclSubmitApps>queueB_user,common_user </aclSubmitApps>");
+    out.println("    <aclAdministerApps>queueB_admin </aclAdministerApps>");
+    out.println("  </queue>");
+    out.println("</queue>");
+    out.println("</allocations>");
+    out.close();
+    fsConf.set(FairSchedulerConfiguration.ALLOCATION_FILE, ALLOC_FILE);
+
+    fsConf.setBoolean(YarnConfiguration.YARN_ACL_ENABLE, true);
+    fsConf.set("yarn.resourcemanager.scheduler.class", FairScheduler.class.getName());
+
+    return fsConf;
+  }
+}

+ 27 - 4
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/apt/FairScheduler.apt.vm

@@ -221,10 +221,14 @@ Allocation file format
      for containers, but apps submitted later may run concurrently if there is
      leftover space on the cluster after satisfying the earlier app's requests.
 
-   * aclSubmitApps: a list of users that can submit apps to the queue. A (default)
-     value of "*" means that any users can submit apps. A queue inherits the ACL of
-     its parent, so if a queue2 descends from queue1, and user1 is in queue1's ACL,
-     and user2 is in queue2's ACL, then both users may submit to queue2.
+   * aclSubmitApps: a list of users and/or groups that can submit apps to the
+     queue. Refer to the ACLs section below for more info on the format of this
+     list and how queue ACLs work.
+
+   * aclAdministerApps: a list of users and/or groups that can administer a
+     queue.  Currently the only administrative action is killing an application.
+     Refer to the ACLs section below for more info on the format of this list
+     and how queue ACLs work.
 
    * minSharePreemptionTimeout: number of seconds the queue is under its minimum share
      before it will try to preempt containers to take resources from other queues.
@@ -246,6 +250,24 @@ Allocation file format
 
   An example allocation file is given here:
 
+Queue Access Control Lists (ACLs)
+
+  Queue Access Control Lists (ACLs) allow administrators to control who may
+  take actions on particular queues. They are configured with the aclSubmitApps
+  and aclAdministerApps properties, which can be set per queue. Currently the
+  only supported administrative action is killing an application. Anybody who
+  may administer a queue may also submit applications to it. These properties
+  take values in a format like "user1,user2 group1,group2" or " group1,group2".
+  An action on a queue will be permitted if its user or group is in the ACL of
+  that queue or in the ACL of any of that queue's ancestors. So if queue2
+  is inside queue1, and user1 is in queue1's ACL, and user2 is in queue2's
+  ACL, then both users may submit to queue2.
+  
+  The root queue's ACLs are "*" by default which, because ACLs are passed down,
+  means that everybody may submit to and kill applications from every queue.
+  To start restricting access, change the root queue's ACLs to something other
+  than "*". 
+
 ---
 <?xml version="1.0"?>
 <allocations>
@@ -256,6 +278,7 @@ Allocation file format
     <weight>2.0</weight>
     <schedulingPolicy>fair</schedulingPolicy>
     <queue name="sample_sub_queue">
+      <aclSubmitApps>charlie</aclSubmitApps>
       <minResources>5000 mb,0vcores</minResources>
     </queue>
   </queue>