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 zh...@apache.org on 2015/03/02 18:15:17 UTC

[01/50] [abbrv] hadoop git commit: HDFS-7467. Provide storage tier information for a directory via fsck. (Benoy Antony)

Repository: hadoop
Updated Branches:
  refs/heads/HDFS-7285 edb292688 -> 22e6b2d3e


HDFS-7467. Provide storage tier information for a directory via fsck. (Benoy Antony)


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

Branch: refs/heads/HDFS-7285
Commit: 7911e1d72e02130ba0f4f0042510ac8b09018ff3
Parents: 11a1c72
Author: Benoy Antony <be...@apache.org>
Authored: Wed Feb 25 16:19:35 2015 -0800
Committer: Zhe Zhang <zh...@cloudera.com>
Committed: Mon Mar 2 09:13:50 2015 -0800

----------------------------------------------------------------------
 .../hdfs/server/namenode/NamenodeFsck.java      |  23 +-
 .../server/namenode/StoragePolicySummary.java   | 257 +++++++++++++++++++
 .../org/apache/hadoop/hdfs/tools/DFSck.java     |   2 +
 .../hadoop/hdfs/server/namenode/TestFsck.java   |  78 +++++-
 .../namenode/TestStoragePolicySummary.java      | 201 +++++++++++++++
 5 files changed, 548 insertions(+), 13 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/7911e1d7/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java
index a3e3a55..f36b773 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java
@@ -25,6 +25,7 @@ import java.net.InetAddress;
 import java.net.InetSocketAddress;
 import java.net.Socket;
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.Collection;
 import java.util.Date;
 import java.util.Iterator;
@@ -45,6 +46,7 @@ import org.apache.hadoop.hdfs.DFSClient;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.RemotePeerFactory;
+import org.apache.hadoop.fs.StorageType;
 import org.apache.hadoop.hdfs.net.Peer;
 import org.apache.hadoop.hdfs.net.TcpPeerServer;
 import org.apache.hadoop.hdfs.protocol.Block;
@@ -128,6 +130,7 @@ public class NamenodeFsck implements DataEncryptionKeyFactory {
   private boolean showBlocks = false;
   private boolean showLocations = false;
   private boolean showRacks = false;
+  private boolean showStoragePolcies = false;
   private boolean showprogress = false;
   private boolean showCorruptFileBlocks = false;
 
@@ -165,6 +168,7 @@ public class NamenodeFsck implements DataEncryptionKeyFactory {
   private List<String> snapshottableDirs = null;
 
   private final BlockPlacementPolicy bpPolicy;
+  private StoragePolicySummary storageTypeSummary = null;
 
   /**
    * Filesystem checker.
@@ -201,6 +205,7 @@ public class NamenodeFsck implements DataEncryptionKeyFactory {
       else if (key.equals("blocks")) { this.showBlocks = true; }
       else if (key.equals("locations")) { this.showLocations = true; }
       else if (key.equals("racks")) { this.showRacks = true; }
+      else if (key.equals("storagepolicies")) { this.showStoragePolcies = true; }
       else if (key.equals("showprogress")) { this.showprogress = true; }
       else if (key.equals("openforwrite")) {this.showOpenFiles = true; }
       else if (key.equals("listcorruptfileblocks")) {
@@ -335,6 +340,11 @@ public class NamenodeFsck implements DataEncryptionKeyFactory {
           return;
         }
         
+        if (this.showStoragePolcies) {
+          storageTypeSummary = new StoragePolicySummary(
+              namenode.getNamesystem().getBlockManager().getStoragePolicies());
+        }
+
         Result res = new Result(conf);
 
         check(path, file, res);
@@ -343,6 +353,10 @@ public class NamenodeFsck implements DataEncryptionKeyFactory {
         out.println(" Number of data-nodes:\t\t" + totalDatanodes);
         out.println(" Number of racks:\t\t" + networktopology.getNumOfRacks());
 
+        if (this.showStoragePolcies) {
+          out.print(storageTypeSummary.toString());
+        }
+
         out.println("FSCK ended at " + new Date() + " in "
             + (Time.now() - startTime + " milliseconds"));
 
@@ -493,7 +507,8 @@ public class NamenodeFsck implements DataEncryptionKeyFactory {
       boolean isCorrupt = lBlk.isCorrupt();
       String blkName = block.toString();
       DatanodeInfo[] locs = lBlk.getLocations();
-      NumberReplicas numberReplicas = namenode.getNamesystem().getBlockManager().countNodes(block.getLocalBlock());
+      NumberReplicas numberReplicas =
+          namenode.getNamesystem().getBlockManager().countNodes(block.getLocalBlock());
       int liveReplicas = numberReplicas.liveReplicas();
       res.totalReplicas += liveReplicas;
       short targetFileReplication = file.getReplication();
@@ -502,6 +517,12 @@ public class NamenodeFsck implements DataEncryptionKeyFactory {
         res.excessiveReplicas += (liveReplicas - targetFileReplication);
         res.numOverReplicatedBlocks += 1;
       }
+      //keep track of storage tier counts
+      if (this.showStoragePolcies && lBlk.getStorageTypes() != null) {
+        StorageType[] storageTypes = lBlk.getStorageTypes();
+        storageTypeSummary.add(Arrays.copyOf(storageTypes, storageTypes.length),
+            fsn.getBlockManager().getStoragePolicy(file.getStoragePolicy()));
+      }
       // Check if block is Corrupt
       if (isCorrupt) {
         corrupt++;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7911e1d7/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/StoragePolicySummary.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/StoragePolicySummary.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/StoragePolicySummary.java
new file mode 100644
index 0000000..bcdad35
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/StoragePolicySummary.java
@@ -0,0 +1,257 @@
+/**
+ * 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.text.NumberFormat;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.EnumMap;
+import java.util.Formatter;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Objects;
+
+import org.apache.hadoop.fs.StorageType;
+import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy;
+
+/**
+ * Aggregate the storage type information for a set of blocks
+ *
+ */
+public class StoragePolicySummary {
+
+  Map<StorageTypeAllocation, Long> storageComboCounts = new HashMap<>();
+  final BlockStoragePolicy[] storagePolicies;
+  int totalBlocks;
+
+  StoragePolicySummary(BlockStoragePolicy[] storagePolicies) {
+    this.storagePolicies = storagePolicies;
+  }
+
+  // Add a storage type combination
+  void add(StorageType[] storageTypes, BlockStoragePolicy policy) {
+    StorageTypeAllocation storageCombo = 
+        new StorageTypeAllocation(storageTypes, policy);
+    Long count = storageComboCounts.get(storageCombo);
+    if (count == null) {
+      storageComboCounts.put(storageCombo, 1l);
+      storageCombo.setActualStoragePolicy(
+          getStoragePolicy(storageCombo.getStorageTypes()));
+    } else {
+      storageComboCounts.put(storageCombo, count.longValue()+1);
+    }
+    totalBlocks++;
+  }
+
+  // sort the storageType combinations based on the total blocks counts
+  // in descending order
+  static List<Entry<StorageTypeAllocation, Long>> sortByComparator(
+      Map<StorageTypeAllocation, Long> unsortMap) {
+    List<Entry<StorageTypeAllocation, Long>> storageAllocations = 
+        new LinkedList<>(unsortMap.entrySet());
+    // Sorting the list based on values
+    Collections.sort(storageAllocations, 
+      new Comparator<Entry<StorageTypeAllocation, Long>>() {
+          public int compare(Entry<StorageTypeAllocation, Long> o1,
+              Entry<StorageTypeAllocation, Long> o2)
+          {
+            return o2.getValue().compareTo(o1.getValue());
+          }
+    });
+    return storageAllocations;
+  }
+
+  public String toString() {
+    StringBuilder compliantBlocksSB = new StringBuilder();
+    compliantBlocksSB.append("\nBlocks satisfying the specified storage policy:");
+    compliantBlocksSB.append("\nStorage Policy                  # of blocks       % of blocks\n");
+    StringBuilder nonCompliantBlocksSB = new StringBuilder();
+    Formatter compliantFormatter = new Formatter(compliantBlocksSB);
+    Formatter nonCompliantFormatter = new Formatter(nonCompliantBlocksSB);
+    NumberFormat percentFormat = NumberFormat.getPercentInstance();
+    percentFormat.setMinimumFractionDigits(4);
+    percentFormat.setMaximumFractionDigits(4);
+    for (Map.Entry<StorageTypeAllocation, Long> storageComboCount:
+      sortByComparator(storageComboCounts)) {
+      double percent = (double) storageComboCount.getValue() / 
+          (double) totalBlocks;
+      StorageTypeAllocation sta = storageComboCount.getKey();
+      if (sta.policyMatches()) {
+        compliantFormatter.format("%-25s %10d  %20s%n",
+            sta.getStoragePolicyDescriptor(),
+            storageComboCount.getValue(),
+            percentFormat.format(percent));
+      } else {
+        if (nonCompliantBlocksSB.length() == 0) {
+          nonCompliantBlocksSB.append("\nBlocks NOT satisfying the specified storage policy:");
+          nonCompliantBlocksSB.append("\nStorage Policy                  ");
+          nonCompliantBlocksSB.append(
+              "Specified Storage Policy      # of blocks       % of blocks\n");
+        }
+        nonCompliantFormatter.format("%-35s %-20s %10d  %20s%n",
+            sta.getStoragePolicyDescriptor(),
+            sta.getSpecifiedStoragePolicy().getName(),
+            storageComboCount.getValue(),
+            percentFormat.format(percent));
+      }
+    }
+    if (nonCompliantBlocksSB.length() == 0) {
+      nonCompliantBlocksSB.append("\nAll blocks satisfy specified storage policy.\n");
+    }
+    compliantFormatter.close();
+    nonCompliantFormatter.close();
+    return compliantBlocksSB.toString() + nonCompliantBlocksSB;
+  }
+
+  /**
+   * 
+   * @param storageTypes - sorted array of storageTypes
+   * @return Storage Policy which matches the specific storage Combination
+   */
+  private BlockStoragePolicy getStoragePolicy(StorageType[] storageTypes) {
+    for (BlockStoragePolicy storagePolicy:storagePolicies) {
+      StorageType[] policyStorageTypes = storagePolicy.getStorageTypes();
+      policyStorageTypes = Arrays.copyOf(policyStorageTypes, policyStorageTypes.length);
+      Arrays.sort(policyStorageTypes);
+      if (policyStorageTypes.length <= storageTypes.length) {
+        int i = 0; 
+        for (; i < policyStorageTypes.length; i++) {
+          if (policyStorageTypes[i] != storageTypes[i]) {
+            break;
+          }
+        }
+        if (i < policyStorageTypes.length) {
+          continue;
+        }
+        int j=policyStorageTypes.length;
+        for (; j < storageTypes.length; j++) {
+          if (policyStorageTypes[i-1] != storageTypes[j]) {
+            break;
+          }
+        }
+
+        if (j==storageTypes.length) {
+          return storagePolicy;
+        }
+      }
+    }
+    return null;
+  }
+
+  /**
+   * Internal class which represents a unique Storage type combination
+   *
+   */
+  static class StorageTypeAllocation {
+    private final BlockStoragePolicy specifiedStoragePolicy;
+    private final StorageType[] storageTypes;
+    private BlockStoragePolicy actualStoragePolicy;
+
+    StorageTypeAllocation(StorageType[] storageTypes, 
+        BlockStoragePolicy specifiedStoragePolicy) {
+      Arrays.sort(storageTypes);
+      this.storageTypes = storageTypes;
+      this.specifiedStoragePolicy = specifiedStoragePolicy;
+    }
+    
+    StorageType[] getStorageTypes() {
+      return storageTypes;
+    }
+
+    BlockStoragePolicy getSpecifiedStoragePolicy() {
+      return specifiedStoragePolicy;
+    }
+    
+    void setActualStoragePolicy(BlockStoragePolicy actualStoragePolicy) {
+      this.actualStoragePolicy = actualStoragePolicy;
+    }
+    
+    BlockStoragePolicy getActualStoragePolicy() {
+      return actualStoragePolicy;
+    }
+
+    private static String getStorageAllocationAsString
+      (Map<StorageType, Integer> storageType_countmap) {
+      StringBuilder sb = new StringBuilder();
+      for (Map.Entry<StorageType, Integer> 
+      storageTypeCountEntry:storageType_countmap.entrySet()) {
+        sb.append(storageTypeCountEntry.getKey().name()+ ":"
+            + storageTypeCountEntry.getValue() + ",");
+      }
+      if (sb.length() > 1) {
+        sb.deleteCharAt(sb.length()-1);
+      }
+      return sb.toString();
+    }
+
+    private String getStorageAllocationAsString() {
+      Map<StorageType, Integer> storageType_countmap = 
+          new EnumMap<>(StorageType.class);
+      for (StorageType storageType: storageTypes) {
+        Integer count = storageType_countmap.get(storageType);
+        if (count == null) {
+          storageType_countmap.put(storageType, 1);
+        } else {
+          storageType_countmap.put(storageType, count.intValue()+1);
+        }
+      }
+      return (getStorageAllocationAsString(storageType_countmap));
+    }
+    
+    String getStoragePolicyDescriptor() {
+      StringBuilder storagePolicyDescriptorSB = new StringBuilder();
+      if (actualStoragePolicy!=null) {
+        storagePolicyDescriptorSB.append(getStorageAllocationAsString())
+        .append("(")
+        .append(actualStoragePolicy.getName())
+        .append(")");
+      } else {
+        storagePolicyDescriptorSB.append(getStorageAllocationAsString());
+      }
+      return storagePolicyDescriptorSB.toString();
+    }
+    
+    boolean policyMatches() {
+      return specifiedStoragePolicy.equals(actualStoragePolicy);
+    }
+    
+    @Override
+    public String toString() {
+      return specifiedStoragePolicy.getName() + "|" + getStoragePolicyDescriptor();
+    }
+
+    @Override
+    public int hashCode() {
+      return Objects.hash(specifiedStoragePolicy,Arrays.hashCode(storageTypes));
+    }
+
+    @Override
+    public boolean equals(Object another) {
+      return (another instanceof StorageTypeAllocation && 
+          Objects.equals(specifiedStoragePolicy,
+              ((StorageTypeAllocation)another).specifiedStoragePolicy) &&
+              Arrays.equals(storageTypes,
+                  ((StorageTypeAllocation)another).storageTypes));
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7911e1d7/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSck.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSck.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSck.java
index 8c934c2..ec83a90 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSck.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSck.java
@@ -93,6 +93,7 @@ public class DFSck extends Configured implements Tool {
       + "\t-blocks\tprint out block report\n"
       + "\t-locations\tprint out locations for every block\n"
       + "\t-racks\tprint out network topology for data-node locations\n"
+      + "\t-storagepolicies\tprint out storage policy summary for the blocks\n"
       + "\t-showprogress\tshow progress in output. Default is OFF (no progress)\n"
       + "\t-blockId\tprint out which file this blockId belongs to, locations"
       + " (nodes, racks) of this block, and other diagnostics info"
@@ -266,6 +267,7 @@ public class DFSck extends Configured implements Tool {
       else if (args[idx].equals("-blocks")) { url.append("&blocks=1"); }
       else if (args[idx].equals("-locations")) { url.append("&locations=1"); }
       else if (args[idx].equals("-racks")) { url.append("&racks=1"); }
+      else if (args[idx].equals("-storagepolicies")) { url.append("&storagepolicies=1"); }
       else if (args[idx].equals("-showprogress")) { url.append("&showprogress=1"); }
       else if (args[idx].equals("-list-corruptfileblocks")) {
         url.append("&listcorruptfileblocks=1");

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7911e1d7/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFsck.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFsck.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFsck.java
index 1941f30..7cdf5ec 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFsck.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFsck.java
@@ -18,6 +18,18 @@
 
 package org.apache.hadoop.hdfs.server.namenode;
 
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+import static org.mockito.Matchers.anyBoolean;
+import static org.mockito.Matchers.anyLong;
+import static org.mockito.Matchers.anyString;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
 import java.io.BufferedReader;
 import java.io.ByteArrayOutputStream;
 import java.io.File;
@@ -41,7 +53,6 @@ import java.util.Set;
 import java.util.regex.Matcher;
 import java.util.regex.Pattern;
 
-import com.google.common.collect.Sets;
 import org.apache.commons.logging.impl.Log4JLogger;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataOutputStream;
@@ -58,6 +69,7 @@ import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.fs.StorageType;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.CorruptFileBlocks;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
@@ -83,17 +95,7 @@ import org.apache.log4j.PatternLayout;
 import org.apache.log4j.RollingFileAppender;
 import org.junit.Test;
 
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertNull;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-import static org.mockito.Matchers.anyBoolean;
-import static org.mockito.Matchers.anyLong;
-import static org.mockito.Matchers.anyString;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.when;
+import com.google.common.collect.Sets;
 
 /**
  * A JUnit test for doing fsck
@@ -1326,4 +1328,56 @@ public class TestFsck {
       }
     }
   }
+
+  private void writeFile(final DistributedFileSystem dfs,
+      Path dir, String fileName) throws IOException {
+    Path filePath = new Path(dir.toString() + Path.SEPARATOR + fileName);
+    final FSDataOutputStream out = dfs.create(filePath);
+    out.writeChars("teststring");
+    out.close();
+  }
+
+  private void writeFile(final DistributedFileSystem dfs,
+      String dirName, String fileName, String StoragePolicy) throws IOException {
+    Path dirPath = new Path(dirName);
+    dfs.mkdirs(dirPath);
+    dfs.setStoragePolicy(dirPath, StoragePolicy);
+    writeFile(dfs, dirPath, fileName);
+  }
+
+  /**
+   * Test storage policy display
+   */
+  @Test
+  public void testStoragePoliciesCK() throws Exception {
+    final Configuration conf = new HdfsConfiguration();
+    final MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf)
+        .numDataNodes(3)
+        .storageTypes(
+            new StorageType[] {StorageType.DISK, StorageType.ARCHIVE})
+        .build();
+    try {
+      cluster.waitActive();
+      final DistributedFileSystem dfs = cluster.getFileSystem();
+      writeFile(dfs, "/testhot", "file", "HOT");
+      writeFile(dfs, "/testwarm", "file", "WARM");
+      writeFile(dfs, "/testcold", "file", "COLD");
+      String outStr = runFsck(conf, 0, true, "/", "-storagepolicies");
+      assertTrue(outStr.contains("DISK:3(HOT)"));
+      assertTrue(outStr.contains("DISK:1,ARCHIVE:2(WARM)"));
+      assertTrue(outStr.contains("ARCHIVE:3(COLD)"));
+      assertTrue(outStr.contains("All blocks satisfy specified storage policy."));
+      dfs.setStoragePolicy(new Path("/testhot"), "COLD");
+      dfs.setStoragePolicy(new Path("/testwarm"), "COLD");
+      outStr = runFsck(conf, 0, true, "/", "-storagepolicies");
+      assertTrue(outStr.contains("DISK:3(HOT)"));
+      assertTrue(outStr.contains("DISK:1,ARCHIVE:2(WARM)"));
+      assertTrue(outStr.contains("ARCHIVE:3(COLD)"));
+      assertFalse(outStr.contains("All blocks satisfy specified storage policy."));
+     } finally {
+      if (cluster != null) {
+        cluster.shutdown();
+      }
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7911e1d7/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStoragePolicySummary.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStoragePolicySummary.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStoragePolicySummary.java
new file mode 100644
index 0000000..60c9318
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStoragePolicySummary.java
@@ -0,0 +1,201 @@
+/**
+ * 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.HashMap;
+import java.util.LinkedHashMap;
+import java.util.Map;
+
+import org.apache.hadoop.fs.StorageType;
+import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite;
+import org.apache.hadoop.hdfs.server.namenode.StoragePolicySummary.StorageTypeAllocation;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class TestStoragePolicySummary {
+  
+  private Map<String, Long> convertToStringMap(StoragePolicySummary sts) {
+    LinkedHashMap<String, Long> actualOutput = new LinkedHashMap<>();
+    for (Map.Entry<StorageTypeAllocation, Long> entry:
+      StoragePolicySummary.sortByComparator(sts.storageComboCounts)) {
+      actualOutput.put(entry.getKey().toString(), entry.getValue());
+    }
+    return actualOutput;
+  }
+
+  @Test
+  public void testMultipleHots() {
+    BlockStoragePolicySuite bsps = BlockStoragePolicySuite.createDefaultSuite();
+    StoragePolicySummary sts = new StoragePolicySummary(bsps.getAllPolicies());
+    BlockStoragePolicy hot = bsps.getPolicy("HOT");
+    sts.add(new StorageType[]{StorageType.DISK},hot);
+    sts.add(new StorageType[]{StorageType.DISK,StorageType.DISK},hot);
+    sts.add(new StorageType[]{StorageType.DISK,
+        StorageType.DISK,StorageType.DISK},hot);
+    sts.add(new StorageType[]{StorageType.DISK,
+        StorageType.DISK,StorageType.DISK,StorageType.DISK},hot);
+    Map<String, Long> actualOutput = convertToStringMap(sts);
+    Assert.assertEquals(4,actualOutput.size());
+    Map<String, Long>  expectedOutput = new HashMap<>();
+    expectedOutput.put("HOT|DISK:1(HOT)", 1l);
+    expectedOutput.put("HOT|DISK:2(HOT)", 1l);
+    expectedOutput.put("HOT|DISK:3(HOT)", 1l);
+    expectedOutput.put("HOT|DISK:4(HOT)", 1l);
+    Assert.assertEquals(expectedOutput,actualOutput);
+  }
+  
+  @Test
+  public void testMultipleHotsWithDifferentCounts() {
+    BlockStoragePolicySuite bsps = BlockStoragePolicySuite.createDefaultSuite();
+    StoragePolicySummary sts = new StoragePolicySummary(bsps.getAllPolicies());
+    BlockStoragePolicy hot = bsps.getPolicy("HOT");
+    sts.add(new StorageType[]{StorageType.DISK},hot);
+    sts.add(new StorageType[]{StorageType.DISK,StorageType.DISK},hot);
+    sts.add(new StorageType[]{StorageType.DISK,StorageType.DISK},hot);
+    sts.add(new StorageType[]{StorageType.DISK,
+        StorageType.DISK,StorageType.DISK},hot);
+    sts.add(new StorageType[]{StorageType.DISK,
+        StorageType.DISK,StorageType.DISK},hot);
+    sts.add(new StorageType[]{StorageType.DISK,
+        StorageType.DISK,StorageType.DISK,StorageType.DISK},hot);
+    Map<String, Long> actualOutput = convertToStringMap(sts);
+    Assert.assertEquals(4,actualOutput.size());
+    Map<String, Long> expectedOutput = new HashMap<>();
+    expectedOutput.put("HOT|DISK:1(HOT)", 1l);
+    expectedOutput.put("HOT|DISK:2(HOT)", 2l);
+    expectedOutput.put("HOT|DISK:3(HOT)", 2l);
+    expectedOutput.put("HOT|DISK:4(HOT)", 1l);
+    Assert.assertEquals(expectedOutput,actualOutput);
+  }
+  
+  @Test
+  public void testMultipleWarmsInDifferentOrder() {
+    BlockStoragePolicySuite bsps = BlockStoragePolicySuite.createDefaultSuite();
+    StoragePolicySummary sts = new StoragePolicySummary(bsps.getAllPolicies());
+    BlockStoragePolicy warm = bsps.getPolicy("WARM");
+    //DISK:1,ARCHIVE:1
+    sts.add(new StorageType[]{StorageType.DISK,StorageType.ARCHIVE},warm);
+    sts.add(new StorageType[]{StorageType.ARCHIVE,StorageType.DISK},warm);
+    //DISK:2,ARCHIVE:1
+    sts.add(new StorageType[]{StorageType.ARCHIVE,
+        StorageType.DISK,StorageType.DISK},warm);
+    sts.add(new StorageType[]{StorageType.DISK,
+        StorageType.ARCHIVE,StorageType.DISK},warm);
+    sts.add(new StorageType[]{StorageType.DISK,
+        StorageType.DISK,StorageType.ARCHIVE},warm);
+    //DISK:1,ARCHIVE:2
+    sts.add(new StorageType[]{StorageType.DISK,
+        StorageType.ARCHIVE,StorageType.ARCHIVE},warm);
+    sts.add(new StorageType[]{StorageType.ARCHIVE,
+        StorageType.DISK,StorageType.ARCHIVE},warm);
+    sts.add(new StorageType[]{StorageType.ARCHIVE,
+        StorageType.ARCHIVE,StorageType.DISK},warm);
+    //DISK:2,ARCHIVE:2
+    sts.add(new StorageType[]{StorageType.ARCHIVE,
+        StorageType.ARCHIVE,StorageType.DISK,StorageType.DISK},warm);
+    Map<String, Long> actualOutput = convertToStringMap(sts);
+    Assert.assertEquals(4,actualOutput.size());
+    Map<String, Long>  expectedOutput = new HashMap<>();
+    expectedOutput.put("WARM|DISK:1,ARCHIVE:1(WARM)", 2l);
+    expectedOutput.put("WARM|DISK:2,ARCHIVE:1", 3l);
+    expectedOutput.put("WARM|DISK:1,ARCHIVE:2(WARM)", 3l);
+    expectedOutput.put("WARM|DISK:2,ARCHIVE:2", 1l);
+    Assert.assertEquals(expectedOutput,actualOutput);
+  }
+  
+  @Test
+  public void testDifferentSpecifiedPolicies() {
+    BlockStoragePolicySuite bsps = BlockStoragePolicySuite.createDefaultSuite();
+    StoragePolicySummary sts = new StoragePolicySummary(bsps.getAllPolicies());
+    BlockStoragePolicy hot = bsps.getPolicy("HOT");
+    BlockStoragePolicy warm = bsps.getPolicy("WARM");
+    BlockStoragePolicy cold = bsps.getPolicy("COLD");
+    //DISK:3
+    sts.add(new StorageType[]{StorageType.DISK,StorageType.DISK,StorageType.DISK},hot);
+    sts.add(new StorageType[]{StorageType.DISK,StorageType.DISK,StorageType.DISK},hot);
+    sts.add(new StorageType[]{StorageType.DISK,StorageType.DISK,StorageType.DISK},warm);
+    sts.add(new StorageType[]{StorageType.DISK,StorageType.DISK,StorageType.DISK},cold);
+    //DISK:1,ARCHIVE:2
+    sts.add(new StorageType[]{StorageType.DISK,
+        StorageType.ARCHIVE,StorageType.ARCHIVE},hot);
+    sts.add(new StorageType[]{StorageType.ARCHIVE,
+        StorageType.DISK,StorageType.ARCHIVE},warm);
+    sts.add(new StorageType[]{StorageType.ARCHIVE,
+        StorageType.ARCHIVE,StorageType.DISK},cold);
+    sts.add(new StorageType[]{StorageType.ARCHIVE,
+        StorageType.ARCHIVE,StorageType.DISK},cold);
+    //ARCHIVE:3
+    sts.add(new StorageType[]{StorageType.ARCHIVE,
+        StorageType.ARCHIVE,StorageType.ARCHIVE},hot);
+    sts.add(new StorageType[]{StorageType.ARCHIVE,
+        StorageType.ARCHIVE,StorageType.ARCHIVE},hot);
+    sts.add(new StorageType[]{StorageType.ARCHIVE,
+        StorageType.ARCHIVE,StorageType.ARCHIVE},warm);
+    sts.add(new StorageType[]{StorageType.ARCHIVE,
+        StorageType.ARCHIVE,StorageType.ARCHIVE},cold);
+    Map<String, Long> actualOutput = convertToStringMap(sts);
+    Assert.assertEquals(9,actualOutput.size());
+    Map<String, Long>  expectedOutput = new HashMap<>();
+    expectedOutput.put("HOT|DISK:3(HOT)", 2l);
+    expectedOutput.put("COLD|DISK:1,ARCHIVE:2(WARM)", 2l);
+    expectedOutput.put("HOT|ARCHIVE:3(COLD)", 2l);
+    expectedOutput.put("WARM|DISK:3(HOT)", 1l);
+    expectedOutput.put("COLD|DISK:3(HOT)", 1l);
+    expectedOutput.put("WARM|ARCHIVE:3(COLD)", 1l);
+    expectedOutput.put("WARM|DISK:1,ARCHIVE:2(WARM)", 1l);
+    expectedOutput.put("COLD|ARCHIVE:3(COLD)", 1l);
+    expectedOutput.put("HOT|DISK:1,ARCHIVE:2(WARM)", 1l);
+    Assert.assertEquals(expectedOutput,actualOutput);
+  }
+  
+  @Test
+  public void testSortInDescendingOrder() {
+    BlockStoragePolicySuite bsps = BlockStoragePolicySuite.createDefaultSuite();
+    StoragePolicySummary sts = new StoragePolicySummary(bsps.getAllPolicies());
+    BlockStoragePolicy hot = bsps.getPolicy("HOT");
+    BlockStoragePolicy warm = bsps.getPolicy("WARM");
+    BlockStoragePolicy cold = bsps.getPolicy("COLD");
+    //DISK:3
+    sts.add(new StorageType[]{StorageType.DISK,StorageType.DISK,StorageType.DISK},hot);
+    sts.add(new StorageType[]{StorageType.DISK,StorageType.DISK,StorageType.DISK},hot);
+    //DISK:1,ARCHIVE:2
+    sts.add(new StorageType[]{StorageType.DISK,
+        StorageType.ARCHIVE,StorageType.ARCHIVE},warm);
+    sts.add(new StorageType[]{StorageType.ARCHIVE,
+        StorageType.DISK,StorageType.ARCHIVE},warm);
+    sts.add(new StorageType[]{StorageType.ARCHIVE,
+        StorageType.ARCHIVE,StorageType.DISK},warm);
+    //ARCHIVE:3
+    sts.add(new StorageType[]{StorageType.ARCHIVE,
+        StorageType.ARCHIVE,StorageType.ARCHIVE},cold);
+    sts.add(new StorageType[]{StorageType.ARCHIVE,
+        StorageType.ARCHIVE,StorageType.ARCHIVE},cold);
+    sts.add(new StorageType[]{StorageType.ARCHIVE,
+        StorageType.ARCHIVE,StorageType.ARCHIVE},cold);
+    sts.add(new StorageType[]{StorageType.ARCHIVE,
+        StorageType.ARCHIVE,StorageType.ARCHIVE},cold);
+    Map<String, Long> actualOutput = convertToStringMap(sts);
+    Assert.assertEquals(3,actualOutput.size());
+    Map<String, Long>  expectedOutput = new LinkedHashMap<>();
+    expectedOutput.put("COLD|ARCHIVE:3(COLD)", 4l);
+    expectedOutput.put("WARM|DISK:1,ARCHIVE:2(WARM)", 3l);
+    expectedOutput.put("HOT|DISK:3(HOT)", 2l);
+    Assert.assertEquals(expectedOutput.toString(),actualOutput.toString());
+  }
+}


[38/50] [abbrv] hadoop git commit: YARN-3199. Fair Scheduler documentation improvements (Rohit Agarwal via aw)

Posted by zh...@apache.org.
YARN-3199. Fair Scheduler documentation improvements (Rohit Agarwal via aw)


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

Branch: refs/heads/HDFS-7285
Commit: e70ce6f917faf4183002939f0842bd71b4073f4f
Parents: 06aca7c
Author: Allen Wittenauer <aw...@apache.org>
Authored: Sat Feb 28 11:36:15 2015 -0800
Committer: Zhe Zhang <zh...@cloudera.com>
Committed: Mon Mar 2 09:13:54 2015 -0800

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                                 | 5 ++++-
 .../hadoop-yarn-site/src/site/markdown/FairScheduler.md         | 2 ++
 2 files changed, 6 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/e70ce6f9/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index 02b1831..cef1758 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -1,6 +1,6 @@
 Hadoop YARN Change Log
 
-Trunk - Unreleased 
+Trunk - Unreleased
 
   INCOMPATIBLE CHANGES
 
@@ -23,6 +23,9 @@ Trunk - Unreleased
     YARN-3168. Convert site documentation from apt to markdown (Gururaj Shetty
     via aw)
 
+    YARN-3199. Fair Scheduler documentation improvements (Rohit Agarwal via
+    aw)
+
   OPTIMIZATIONS
 
   BUG FIXES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e70ce6f9/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/FairScheduler.md
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/FairScheduler.md b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/FairScheduler.md
index 1812a44..a58b3d3 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/FairScheduler.md
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/FairScheduler.md
@@ -85,6 +85,8 @@ Customizing the Fair Scheduler typically involves altering two files. First, sch
 | `yarn.scheduler.fair.locality.threshold.rack` | For applications that request containers on particular racks, the number of scheduling opportunities since the last container assignment to wait before accepting a placement on another rack. Expressed as a float between 0 and 1, which, as a fraction of the cluster size, is the number of scheduling opportunities to pass up. The default value of -1.0 means don't pass up any scheduling opportunities. |
 | `yarn.scheduler.fair.allow-undeclared-pools` | If this is true, new queues can be created at application submission time, whether because they are specified as the application's queue by the submitter or because they are placed there by the user-as-default-queue property. If this is false, any time an app would be placed in a queue that is not specified in the allocations file, it is placed in the "default" queue instead. Defaults to true. If a queue placement policy is given in the allocations file, this property is ignored. |
 | `yarn.scheduler.fair.update-interval-ms` | The interval at which to lock the scheduler and recalculate fair shares, recalculate demand, and check whether anything is due for preemption. Defaults to 500 ms. |
+| `yarn.scheduler.increment-allocation-mb` | The fairscheduler grants memory in increments of this value. If you submit a task with resource request that is not a multiple of increment-allocation-mb, the request will be rounded up to the nearest increment. Defaults to 1024 MB. |
+| `yarn.scheduler.increment-allocation-vcores` | The fairscheduler grants vcores in increments of this value. If you submit a task with resource request that is not a multiple of increment-allocation-vcores, the request will be rounded up to the nearest increment. Defaults to 1. |
 
 ###Allocation file format
 


[36/50] [abbrv] hadoop git commit: YARN-3168. Convert site documentation from apt to markdown (Gururaj Shetty via aw)

Posted by zh...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/06aca7c6/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/apt/NodeManagerRest.apt.vm
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/apt/NodeManagerRest.apt.vm b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/apt/NodeManagerRest.apt.vm
deleted file mode 100644
index 36b8621..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/apt/NodeManagerRest.apt.vm
+++ /dev/null
@@ -1,645 +0,0 @@
-~~ 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.
-
-  ---
-  NodeManager REST API's.
-  ---
-  ---
-  ${maven.build.timestamp}
-
-NodeManager REST API's.
-
-%{toc|section=1|fromDepth=0|toDepth=2}
-
-* Overview
-
-  The NodeManager REST API's allow the user to get status on the node and information about applications and containers running on that node. 
-  
-* NodeManager Information API
-
-  The node information resource provides overall information about that particular node.
-
-** URI
-
-  Both of the following URI's give you the cluster information.
-
-------
-  * http://<nm http address:port>/ws/v1/node
-  * http://<nm http address:port>/ws/v1/node/info
-------
-
-** HTTP Operations Supported
-
-------
-  * GET
-------
-
-** Query Parameters Supported
-
-------
-  None
-------
-
-** Elements of the <nodeInfo> object
-
-*---------------+--------------+-------------------------------+
-|| Item         || Data Type   || Description                   |
-*---------------+--------------+-------------------------------+
-| id            | long         | The NodeManager id |
-*---------------+--------------+-------------------------------+
-| nodeHostName | string  | The host name of the NodeManager |
-*---------------+--------------+-------------------------------+
-| totalPmemAllocatedContainersMB | long         | The amount of physical memory allocated for use by containers in MB |
-*---------------+--------------+-------------------------------+
-| totalVmemAllocatedContainersMB | long         | The amount of virtual memory allocated for use by containers in MB |
-*---------------+--------------+-------------------------------+
-| totalVCoresAllocatedContainers | long         | The number of virtual cores allocated for use by containers |
-*---------------+--------------+-------------------------------+
-| lastNodeUpdateTime | long         | The last timestamp at which the health report was received (in ms since epoch)|
-*---------------+--------------+-------------------------------+
-| healthReport | string  | The diagnostic health report of the node |
-*---------------+--------------+-------------------------------+
-| nodeHealthy | boolean | true/false indicator of if the node is healthy|
-*---------------+--------------+-------------------------------+
-| nodeManagerVersion | string  | Version of the NodeManager |
-*---------------+--------------+-------------------------------+
-| nodeManagerBuildVersion | string  | NodeManager build string with build version, user, and checksum |
-*---------------+--------------+-------------------------------+
-| nodeManagerVersionBuiltOn | string  | Timestamp when NodeManager was built(in ms since epoch) |
-*---------------+--------------+-------------------------------+
-| hadoopVersion | string  | Version of hadoop common |
-*---------------+--------------+-------------------------------+
-| hadoopBuildVersion | string  | Hadoop common build string with build version, user, and checksum |
-*---------------+--------------+-------------------------------+
-| hadoopVersionBuiltOn | string  | Timestamp when hadoop common was built(in ms since epoch) |
-*---------------+--------------+-------------------------------+
-
-** Response Examples
-
-  <<JSON response>>
-
-  HTTP Request:
-
-------
-  GET http://<nm http address:port>/ws/v1/node/info
-------
-
-  Response Header:
-
-+---+
-  HTTP/1.1 200 OK
-  Content-Type: application/json
-  Transfer-Encoding: chunked
-  Server: Jetty(6.1.26)
-+---+
-
-  Response Body:
-
-+---+
-{
-   "nodeInfo" : {
-      "hadoopVersionBuiltOn" : "Mon Jan  9 14:58:42 UTC 2012",
-      "nodeManagerBuildVersion" : "0.23.1-SNAPSHOT from 1228355 by user1 source checksum 20647f76c36430e888cc7204826a445c",
-      "lastNodeUpdateTime" : 1326222266126,
-      "totalVmemAllocatedContainersMB" : 17203,
-      "totalVCoresAllocatedContainers" : 8,
-      "nodeHealthy" : true,
-      "healthReport" : "",
-      "totalPmemAllocatedContainersMB" : 8192,
-      "nodeManagerVersionBuiltOn" : "Mon Jan  9 15:01:59 UTC 2012",
-      "nodeManagerVersion" : "0.23.1-SNAPSHOT",
-      "id" : "host.domain.com:8041",
-      "hadoopBuildVersion" : "0.23.1-SNAPSHOT from 1228292 by user1 source checksum 3eba233f2248a089e9b28841a784dd00",
-      "nodeHostName" : "host.domain.com",
-      "hadoopVersion" : "0.23.1-SNAPSHOT"
-   }
-}
-+---+
-
-  <<XML response>>
-
-  HTTP Request:
-
------
-  Accept: application/xml
-  GET http://<nm http address:port>/ws/v1/node/info
------
-
-  Response Header:
-
-+---+
-  HTTP/1.1 200 OK
-  Content-Type: application/xml
-  Content-Length: 983
-  Server: Jetty(6.1.26)
-+---+
-
-  Response Body:
-
-+---+
-<?xml version="1.0" encoding="UTF-8" standalone="yes"?>
-<nodeInfo>
-  <healthReport/>
-  <totalVmemAllocatedContainersMB>17203</totalVmemAllocatedContainersMB>
-  <totalPmemAllocatedContainersMB>8192</totalPmemAllocatedContainersMB>
-  <totalVCoresAllocatedContainers>8</totalVCoresAllocatedContainers>
-  <lastNodeUpdateTime>1326222386134</lastNodeUpdateTime>
-  <nodeHealthy>true</nodeHealthy>
-  <nodeManagerVersion>0.23.1-SNAPSHOT</nodeManagerVersion>
-  <nodeManagerBuildVersion>0.23.1-SNAPSHOT from 1228355 by user1 source checksum 20647f76c36430e888cc7204826a445c</nodeManagerBuildVersion>
-  <nodeManagerVersionBuiltOn>Mon Jan  9 15:01:59 UTC 2012</nodeManagerVersionBuiltOn>
-  <hadoopVersion>0.23.1-SNAPSHOT</hadoopVersion>
-  <hadoopBuildVersion>0.23.1-SNAPSHOT from 1228292 by user1 source checksum 3eba233f2248a089e9b28841a784dd00</hadoopBuildVersion>
-  <hadoopVersionBuiltOn>Mon Jan  9 14:58:42 UTC 2012</hadoopVersionBuiltOn>
-  <id>host.domain.com:8041</id>
-  <nodeHostName>host.domain.com</nodeHostName>
-</nodeInfo>
-+---+
-
-* Applications API
-
-  With the Applications API, you can obtain a collection of resources, each of which represents an application.  When you run a GET operation on this resource, you obtain a collection of Application Objects.  See also {{Application API}} for syntax of the application object.
-
-** URI
-
-------
-  * http://<nm http address:port>/ws/v1/node/apps
-------
-
-** HTTP Operations Supported 
-
-------
-  * GET
-------
-
-** Query Parameters Supported
-
-  Multiple paramters can be specified.  
-
-------
-  * state - application state 
-  * user - user name
-------
-
-** Elements of the <apps> (Applications) object
-
-  When you make a request for the list of applications, the information will be returned as a collection of app objects. 
-  See also {{Application API}} for syntax of the app object.
-
-*---------------+--------------+-------------------------------+
-|| Item         || Data Type   || Description                  |
-*---------------+--------------+-------------------------------+
-| app | array of app objects(JSON)/zero or more app objects(XML) | A collection of application objects |
-*---------------+--------------+--------------------------------+
-
-** Response Examples
-
-  <<JSON response>>
-
-  HTTP Request:
-
-------
-  GET http://<nm http address:port>/ws/v1/node/apps
-------
-
-  Response Header:
-
-+---+
-  HTTP/1.1 200 OK
-  Content-Type: application/json
-  Transfer-Encoding: chunked
-  Server: Jetty(6.1.26)
-+---+
-
-  Response Body:
-
-+---+
-{
-   "apps" : {
-      "app" : [
-         {
-            "containerids" : [
-               "container_1326121700862_0003_01_000001",
-               "container_1326121700862_0003_01_000002"
-            ],
-            "user" : "user1",
-            "id" : "application_1326121700862_0003",
-            "state" : "RUNNING"
-         },
-         {
-            "user" : "user1",
-            "id" : "application_1326121700862_0002",
-            "state" : "FINISHED"
-         }
-      ]
-   }
-}
-+---+
-
-  <<XML response>>
-
-  HTTP Request:
-
-------
-  GET http://<nm http address:port>/ws/v1/node/apps
-  Accept: application/xml
-------
-
-  Response Header:
-
-+---+
-  HTTP/1.1 200 OK
-  Content-Type: application/xml
-  Content-Length: 400
-  Server: Jetty(6.1.26)
-+---+
-
-  Response Body:
-
-+---+
-<?xml version="1.0" encoding="UTF-8" standalone="yes"?>
-<apps>
-  <app>
-    <id>application_1326121700862_0002</id>
-    <state>FINISHED</state>
-    <user>user1</user>
-  </app>
-  <app>
-    <id>application_1326121700862_0003</id>
-    <state>RUNNING</state>
-    <user>user1</user>
-    <containerids>container_1326121700862_0003_01_000002</containerids>
-    <containerids>container_1326121700862_0003_01_000001</containerids>
-  </app>
-</apps>
-
-+---+
-
-* {Application API}
-
-  An application resource contains information about a particular application that was run or is running on this NodeManager.
-
-** URI
-
-  Use the following URI to obtain an app Object, for a application identified by the {appid} value. 
-
-------
-  * http://<nm http address:port>/ws/v1/node/apps/{appid}
-------
-
-** HTTP Operations Supported 
-
-------
-  * GET
-------
-
-** Query Parameters Supported
-
-------
-  None
-------
-
-** Elements of the <app> (Application) object
-
-*---------------+--------------+-------------------------------+
-|| Item         || Data Type   || Description                  |
-*---------------+--------------+-------------------------------+
-| id | string  | The application id | 
-*---------------+--------------+--------------------------------+
-| user | string  | The user who started the application |
-*---------------+--------------+--------------------------------+
-| state | string | The state of the application -  valid states are: NEW, INITING, RUNNING, FINISHING_CONTAINERS_WAIT, APPLICATION_RESOURCES_CLEANINGUP, FINISHED |
-*---------------+--------------+--------------------------------+
-| containerids | array of containerids(JSON)/zero or more containerids(XML) | The list of containerids currently being used by the application on this node. If not present then no containers are currently running for this application.|
-*---------------+--------------+--------------------------------+
-
-** Response Examples
-
-  <<JSON response>>
-
-  HTTP Request:
-
-------
-  GET http://<nm http address:port>/ws/v1/node/apps/application_1326121700862_0005
-------
-
-  Response Header:
-
-+---+
-  HTTP/1.1 200 OK
-  Content-Type: application/json
-  Transfer-Encoding: chunked
-  Server: Jetty(6.1.26)
-+---+
-
-  Response Body:
-
-+---+
-{
-   "app" : {
-      "containerids" : [
-         "container_1326121700862_0005_01_000003",
-         "container_1326121700862_0005_01_000001"
-      ],
-      "user" : "user1",
-      "id" : "application_1326121700862_0005",
-      "state" : "RUNNING"
-   }
-}
-+---+
-
-  <<XML response>>
-
-  HTTP Request:
-
-------
-  GET http://<nm http address:port>/ws/v1/node/apps/application_1326121700862_0005
-  Accept: application/xml
-------
-
-  Response Header:
-
-+---+
-  HTTP/1.1 200 OK
-  Content-Type: application/xml
-  Content-Length: 281 
-  Server: Jetty(6.1.26)
-+---+
-
-  Response Body:
-
-+---+
-<?xml version="1.0" encoding="UTF-8" standalone="yes"?>
-<app>
-  <id>application_1326121700862_0005</id>
-  <state>RUNNING</state>
-  <user>user1</user>
-  <containerids>container_1326121700862_0005_01_000003</containerids>
-  <containerids>container_1326121700862_0005_01_000001</containerids>
-</app>
-+---+
-
-
-* Containers API
-
-  With the containers API, you can obtain a collection of resources, each of which  represents a container.  When you run a GET operation on this resource, you obtain a collection of Container Objects. See also {{Container API}} for syntax of the container object.
-
-** URI
-
-------
-  * http://<nm http address:port>/ws/v1/node/containers
-------
-
-** HTTP Operations Supported 
-
-------
-  * GET
-------
-
-** Query Parameters Supported
-
-------
-  None
-------
-
-** Elements of the <containers> object
-
-  When you make a request for the list of containers, the information will be returned as collection of container objects. 
-  See also {{Container API}} for syntax of the container object.
-
-*---------------+--------------+-------------------------------+
-|| Item         || Data Type   || Description                   |
-*---------------+--------------+-------------------------------+
-| containers | array of container objects(JSON)/zero or more container objects(XML) | A collection of container objects |
-*---------------+--------------+-------------------------------+
-
-** Response Examples
-
-  <<JSON response>>
-
-  HTTP Request:
-
-------
-  GET http://<nm http address:port>/ws/v1/node/containers
-------
-
-  Response Header:
-
-+---+
-  HTTP/1.1 200 OK
-  Content-Type: application/json
-  Transfer-Encoding: chunked
-  Server: Jetty(6.1.26)
-+---+
-
-  Response Body:
-
-+---+
-{
-   "containers" : {
-      "container" : [
-         {
-            "nodeId" : "host.domain.com:8041",
-            "totalMemoryNeededMB" : 2048,
-            "totalVCoresNeeded" : 1,
-            "state" : "RUNNING",
-            "diagnostics" : "",
-            "containerLogsLink" : "http://host.domain.com:8042/node/containerlogs/container_1326121700862_0006_01_000001/user1",
-            "user" : "user1",
-            "id" : "container_1326121700862_0006_01_000001",
-            "exitCode" : -1000
-         },
-         {
-            "nodeId" : "host.domain.com:8041",
-            "totalMemoryNeededMB" : 2048,
-            "totalVCoresNeeded" : 2,
-            "state" : "RUNNING",
-            "diagnostics" : "",
-            "containerLogsLink" : "http://host.domain.com:8042/node/containerlogs/container_1326121700862_0006_01_000003/user1",
-            "user" : "user1",
-            "id" : "container_1326121700862_0006_01_000003",
-            "exitCode" : -1000
-         }
-      ]
-   }
-}
-+---+
-
-  <<XML response>>
-
-  HTTP Request:
-
-------
-  GET http://<nm http address:port>/ws/v1/node/containers
-  Accept: application/xml
-------
-
-  Response Header:
-
-+---+
-  HTTP/1.1 200 OK
-  Content-Type: application/xml
-  Content-Length: 988
-  Server: Jetty(6.1.26)
-+---+
-
-  Response Body:
-
-+---+
-<?xml version="1.0" encoding="UTF-8" standalone="yes"?>
-<containers>
-  <container>
-    <id>container_1326121700862_0006_01_000001</id>
-    <state>RUNNING</state>
-    <exitCode>-1000</exitCode>
-    <diagnostics/>
-    <user>user1</user>
-    <totalMemoryNeededMB>2048</totalMemoryNeededMB>
-    <totalVCoresNeeded>1</totalVCoresNeeded>
-    <containerLogsLink>http://host.domain.com:8042/node/containerlogs/container_1326121700862_0006_01_000001/user1</containerLogsLink>
-    <nodeId>host.domain.com:8041</nodeId>
-  </container>
-  <container>
-    <id>container_1326121700862_0006_01_000003</id>
-    <state>DONE</state>
-    <exitCode>0</exitCode>
-    <diagnostics>Container killed by the ApplicationMaster.</diagnostics>
-    <user>user1</user>
-    <totalMemoryNeededMB>2048</totalMemoryNeededMB>
-    <totalVCoresNeeded>2</totalVCoresNeeded>
-    <containerLogsLink>http://host.domain.com:8042/node/containerlogs/container_1326121700862_0006_01_000003/user1</containerLogsLink>
-    <nodeId>host.domain.com:8041</nodeId>
-  </container>
-</containers>
-+---+
-
-
-* {Container API}
-
-  A container resource contains information about a particular container that is running on this NodeManager.
-
-** URI
-
-  Use the following URI to obtain a Container Object, from a container identified by the {containerid} value. 
-
-------
-  * http://<nm http address:port>/ws/v1/node/containers/{containerid}
-------
-
-** HTTP Operations Supported 
-
-------
-  * GET
-------
-
-** Query Parameters Supported
-
-------
-  None
-------
-
-** Elements of the <container> object
-
-*---------------+--------------+-------------------------------+
-|| Item         || Data Type   || Description                   |
-*---------------+--------------+-------------------------------+
-| id | string  | The container id |
-*---------------+--------------+-------------------------------+
-| state | string | State of the container - valid states are: NEW, LOCALIZING, LOCALIZATION_FAILED, LOCALIZED, RUNNING, EXITED_WITH_SUCCESS, EXITED_WITH_FAILURE, KILLING, CONTAINER_CLEANEDUP_AFTER_KILL, CONTAINER_RESOURCES_CLEANINGUP, DONE|
-*---------------+--------------+-------------------------------+
-| nodeId | string  | The id of the node the container is on|
-*---------------+--------------+-------------------------------+
-| containerLogsLink | string  | The http link to the container logs |
-*---------------+--------------+-------------------------------+
-| user | string  | The user name of the user which started the container|
-*---------------+--------------+-------------------------------+
-| exitCode | int | Exit code of the container |
-*---------------+--------------+-------------------------------+
-| diagnostics | string | A diagnostic message for failed containers |
-*---------------+--------------+-------------------------------+
-| totalMemoryNeededMB | long | Total amout of memory needed by the container (in MB) |
-*---------------+--------------+-------------------------------+
-| totalVCoresNeeded | long | Total number of virtual cores needed by the container |
-*---------------+--------------+-------------------------------+
-
-** Response Examples
-
-  <<JSON response>>
-
-  HTTP Request:
-
-------
-  GET http://<nm http address:port>/ws/v1/nodes/containers/container_1326121700862_0007_01_000001
-------
-
-  Response Header:
-
-+---+
-  HTTP/1.1 200 OK
-  Content-Type: application/json
-  Transfer-Encoding: chunked
-  Server: Jetty(6.1.26)
-+---+
-
-  Response Body:
-
-+---+
-{
-   "container" : {
-      "nodeId" : "host.domain.com:8041",
-      "totalMemoryNeededMB" : 2048,
-      "totalVCoresNeeded" : 1,
-      "state" : "RUNNING",
-      "diagnostics" : "",
-      "containerLogsLink" : "http://host.domain.com:8042/node/containerlogs/container_1326121700862_0007_01_000001/user1",
-      "user" : "user1",
-      "id" : "container_1326121700862_0007_01_000001",
-      "exitCode" : -1000
-   }
-}
-+---+
-
-  <<XML response>>
-
-  HTTP Request:
-
-------
-  GET http://<nm http address:port>/ws/v1/node/containers/container_1326121700862_0007_01_000001
-  Accept: application/xml
-------
-
-  Response Header:
-
-+---+
-  HTTP/1.1 200 OK
-  Content-Type: application/xml
-  Content-Length: 491 
-  Server: Jetty(6.1.26)
-+---+
-
-  Response Body:
-
-+---+
-<?xml version="1.0" encoding="UTF-8" standalone="yes"?>
-<container>
-  <id>container_1326121700862_0007_01_000001</id>
-  <state>RUNNING</state>
-  <exitCode>-1000</exitCode>
-  <diagnostics/>
-  <user>user1</user>
-  <totalMemoryNeededMB>2048</totalMemoryNeededMB>
-  <totalVCoresNeeded>1</totalVCoresNeeded>
-  <containerLogsLink>http://host.domain.com:8042/node/containerlogs/container_1326121700862_0007_01_000001/user1</containerLogsLink>
-  <nodeId>host.domain.com:8041</nodeId>
-</container>
-+---+
-

http://git-wip-us.apache.org/repos/asf/hadoop/blob/06aca7c6/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/apt/NodeManagerRestart.apt.vm
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/apt/NodeManagerRestart.apt.vm b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/apt/NodeManagerRestart.apt.vm
deleted file mode 100644
index ba03f4e..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/apt/NodeManagerRestart.apt.vm
+++ /dev/null
@@ -1,86 +0,0 @@
-~~ 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.
-
-  ---
-  NodeManager Restart
-  ---
-  ---
-  ${maven.build.timestamp}
-
-NodeManager Restart
-
-* Introduction
-
-  This document gives an overview of NodeManager (NM) restart, a feature that
-  enables the NodeManager to be restarted without losing 
-  the active containers running on the node. At a high level, the NM stores any 
-  necessary state to a local state-store as it processes container-management
-  requests. When the NM restarts, it recovers by first loading state for
-  various subsystems and then letting those subsystems perform recovery using
-  the loaded state.
-
-* Enabling NM Restart
-
-  [[1]] To enable NM Restart functionality, set the following property in <<conf/yarn-site.xml>> to true:
-
-*--------------------------------------+--------------------------------------+
-|| Property                            || Value                                |
-*--------------------------------------+--------------------------------------+
-| <<<yarn.nodemanager.recovery.enabled>>> | |
-| | <<<true>>>, (default value is set to false) |
-*--------------------------------------+--------------------------------------+ 
-
-  [[2]] Configure a path to the local file-system directory where the
-  NodeManager can save its run state
-
-*--------------------------------------+--------------------------------------+
-|| Property                            || Description                        |
-*--------------------------------------+--------------------------------------+
-| <<<yarn.nodemanager.recovery.dir>>> | |
-| | The local filesystem directory in which the node manager will store state |
-| | when recovery is enabled.  |
-| | The default value is set to |
-| | <<<${hadoop.tmp.dir}/yarn-nm-recovery>>>. |
-*--------------------------------------+--------------------------------------+ 
-
-  [[3]] Configure a valid RPC address for the NodeManager
-  
-*--------------------------------------+--------------------------------------+
-|| Property                            || Description                        |
-*--------------------------------------+--------------------------------------+
-| <<<yarn.nodemanager.address>>> | |
-| |   Ephemeral ports (port 0, which is default) cannot be used for the |
-| | NodeManager's RPC server specified via yarn.nodemanager.address as it can |
-| | make NM use different ports before and after a restart. This will break any |
-| | previously running clients that were communicating with the NM before |
-| | restart. Explicitly setting yarn.nodemanager.address to an address with |
-| | specific port number (for e.g 0.0.0.0:45454) is a precondition for enabling |
-| | NM restart. |
-*--------------------------------------+--------------------------------------+
-
-  [[4]] Auxiliary services
-  
-  NodeManagers in a YARN cluster can be configured to run auxiliary services.
-  For a completely functional NM restart, YARN relies on any auxiliary service
-  configured to also support recovery. This usually includes (1) avoiding usage
-  of ephemeral ports so that previously running clients (in this case, usually
-  containers) are not disrupted after restart and (2) having the auxiliary
-  service itself support recoverability by reloading any previous state when
-  NodeManager restarts and reinitializes the auxiliary service.
-  
-  A simple example for the above is the auxiliary service 'ShuffleHandler' for
-  MapReduce (MR). ShuffleHandler respects the above two requirements already,
-  so users/admins don't have do anything for it to support NM restart: (1) The
-  configuration property <<mapreduce.shuffle.port>> controls which port the
-  ShuffleHandler on a NodeManager host binds to, and it defaults to a
-  non-ephemeral port. (2) The ShuffleHandler service also already supports
-  recovery of previous state after NM restarts.
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/06aca7c6/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/apt/ResourceManagerHA.apt.vm
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/apt/ResourceManagerHA.apt.vm b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/apt/ResourceManagerHA.apt.vm
deleted file mode 100644
index 0346cda..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/apt/ResourceManagerHA.apt.vm
+++ /dev/null
@@ -1,233 +0,0 @@
-~~ 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.
-
-  ---
-  ResourceManager High Availability
-  ---
-  ---
-  ${maven.build.timestamp}
-
-ResourceManager High Availability
-
-%{toc|section=1|fromDepth=0}
-
-* Introduction
-
-  This guide provides an overview of High Availability of YARN's ResourceManager,
-  and details how to configure and use this feature. The ResourceManager (RM)
-  is responsible for tracking the resources in a cluster, and scheduling
-  applications (e.g., MapReduce jobs). Prior to Hadoop 2.4, the ResourceManager
-  is the single point of failure in a YARN cluster. The High Availability
-  feature adds redundancy in the form of an Active/Standby ResourceManager pair
-  to remove this otherwise single point of failure.
-
-* Architecture
-
-[images/rm-ha-overview.png] Overview of ResourceManager High Availability
-
-** RM Failover
-
-  ResourceManager HA is realized through an Active/Standby architecture - at
-  any point of time, one of the RMs is Active, and one or more RMs are in
-  Standby mode waiting to take over should anything happen to the Active.
-  The trigger to transition-to-active comes from either the admin (through CLI)
-  or through the integrated failover-controller when automatic-failover is
-  enabled.
-
-*** Manual transitions and failover
-
-    When automatic failover is not enabled, admins have to manually transition
-    one of the RMs to Active. To failover from one RM to the other, they are
-    expected to first transition the Active-RM to Standby and transition a
-    Standby-RM to Active. All this can be done using the "<<<yarn rmadmin>>>"
-    CLI.
-
-*** Automatic failover
-
-    The RMs have an option to embed the Zookeeper-based ActiveStandbyElector to
-    decide which RM should be the Active. When the Active goes down or becomes
-    unresponsive, another RM is automatically elected to be the Active which
-    then takes over. Note that, there is no need to run a separate ZKFC daemon
-    as is the case for HDFS because ActiveStandbyElector embedded in RMs acts
-    as a failure detector and a leader elector instead of a separate ZKFC
-    deamon.
-
-*** Client, ApplicationMaster and NodeManager on RM failover
-
-    When there are multiple RMs, the configuration (yarn-site.xml) used by
-    clients and nodes is expected to list all the RMs. Clients,
-    ApplicationMasters (AMs) and NodeManagers (NMs) try connecting to the RMs in
-    a round-robin fashion until they hit the Active RM. If the Active goes down,
-    they resume the round-robin polling until they hit the "new" Active.
-    This default retry logic is implemented as
-    <<<org.apache.hadoop.yarn.client.ConfiguredRMFailoverProxyProvider>>>.
-    You can override the logic by
-    implementing <<<org.apache.hadoop.yarn.client.RMFailoverProxyProvider>>> and
-    setting the value of <<<yarn.client.failover-proxy-provider>>> to
-    the class name.
-
-** Recovering prevous active-RM's state
-
-   With the {{{./ResourceManagerRestart.html}ResourceManger Restart}} enabled,
-   the RM being promoted to an active state loads the RM internal state and
-   continues to operate from where the previous active left off as much as
-   possible depending on the RM restart feature. A new attempt is spawned for
-   each managed application previously submitted to the RM. Applications can
-   checkpoint periodically to avoid losing any work. The state-store must be
-   visible from the both of Active/Standby RMs. Currently, there are two
-   RMStateStore implementations for persistence - FileSystemRMStateStore
-   and ZKRMStateStore.  The <<<ZKRMStateStore>>> implicitly allows write access
-   to a single RM at any point in time, and hence is the recommended store to
-   use in an HA cluster. When using the ZKRMStateStore, there is no need for a
-   separate fencing mechanism to address a potential split-brain situation
-   where multiple RMs can potentially assume the Active role.
-
-
-* Deployment
-
-** Configurations
-
-   Most of the failover functionality is tunable using various configuration
-   properties. Following is a list of required/important ones. yarn-default.xml
-   carries a full-list of knobs. See
-   {{{../hadoop-yarn-common/yarn-default.xml}yarn-default.xml}}
-   for more information including default values.
-   See {{{./ResourceManagerRestart.html}the document for ResourceManger
-   Restart}} also for instructions on setting up the state-store.
-
-*-------------------------+----------------------------------------------+
-|| Configuration Property || Description                                 |
-*-------------------------+----------------------------------------------+
-| yarn.resourcemanager.zk-address | |
-| | Address of the ZK-quorum.
-| | Used both for the state-store and embedded leader-election.
-*-------------------------+----------------------------------------------+
-| yarn.resourcemanager.ha.enabled | |
-| | Enable RM HA
-*-------------------------+----------------------------------------------+
-| yarn.resourcemanager.ha.rm-ids | |
-| | List of logical IDs for the RMs. |
-| | e.g., "rm1,rm2" |
-*-------------------------+----------------------------------------------+
-| yarn.resourcemanager.hostname.<rm-id> | |
-| | For each <rm-id>, specify the hostname the |
-| | RM corresponds to. Alternately, one could set each of the RM's service |
-| | addresses. |
-*-------------------------+----------------------------------------------+
-| yarn.resourcemanager.ha.id | |
-| | Identifies the RM in the ensemble. This is optional; |
-| | however, if set, admins have to ensure that all the RMs have their own |
-| | IDs in the config |
-*-------------------------+----------------------------------------------+
-| yarn.resourcemanager.ha.automatic-failover.enabled | |
-| | Enable automatic failover; |
-| | By default, it is enabled only when HA is enabled. |
-*-------------------------+----------------------------------------------+
-| yarn.resourcemanager.ha.automatic-failover.embedded | |
-| | Use embedded leader-elector |
-| | to pick the Active RM, when automatic failover is enabled. By default, |
-| | it is enabled only when HA is enabled. |
-*-------------------------+----------------------------------------------+
-| yarn.resourcemanager.cluster-id | |
-| | Identifies the cluster. Used by the elector to |
-| | ensure an RM doesn't take over as Active for another cluster. |
-*-------------------------+----------------------------------------------+
-| yarn.client.failover-proxy-provider | |
-| | The class to be used by Clients, AMs and NMs to failover to the Active RM. |
-*-------------------------+----------------------------------------------+
-| yarn.client.failover-max-attempts | |
-| | The max number of times FailoverProxyProvider should attempt failover. |
-*-------------------------+----------------------------------------------+
-| yarn.client.failover-sleep-base-ms | |
-| | The sleep base (in milliseconds) to be used for calculating |
-| | the exponential delay between failovers. |
-*-------------------------+----------------------------------------------+
-| yarn.client.failover-sleep-max-ms | |
-| | The maximum sleep time (in milliseconds) between failovers |
-*-------------------------+----------------------------------------------+
-| yarn.client.failover-retries | |
-| | The number of retries per attempt to connect to a ResourceManager. |
-*-------------------------+----------------------------------------------+
-| yarn.client.failover-retries-on-socket-timeouts | |
-| | The number of retries per attempt to connect to a ResourceManager on socket timeouts. |
-*-------------------------+----------------------------------------------+
-
-*** Sample configurations
-
-  Here is the sample of minimal setup for RM failover.
-
-+---+
- <property>
-   <name>yarn.resourcemanager.ha.enabled</name>
-   <value>true</value>
- </property>
- <property>
-   <name>yarn.resourcemanager.cluster-id</name>
-   <value>cluster1</value>
- </property>
- <property>
-   <name>yarn.resourcemanager.ha.rm-ids</name>
-   <value>rm1,rm2</value>
- </property>
- <property>
-   <name>yarn.resourcemanager.hostname.rm1</name>
-   <value>master1</value>
- </property>
- <property>
-   <name>yarn.resourcemanager.hostname.rm2</name>
-   <value>master2</value>
- </property>
- <property>
-   <name>yarn.resourcemanager.zk-address</name>
-   <value>zk1:2181,zk2:2181,zk3:2181</value>
- </property>
-+---+
-
-** Admin commands
-
-   <<<yarn rmadmin>>> has a few HA-specific command options to check the health/state of an
-   RM, and transition to Active/Standby.
-   Commands for HA take service id of RM set by <<<yarn.resourcemanager.ha.rm-ids>>>
-   as argument.
-
-+---+
- $ yarn rmadmin -getServiceState rm1
- active
- 
- $ yarn rmadmin -getServiceState rm2
- standby
-+---+
-
-   If automatic failover is enabled, you can not use manual transition command.
-   Though you can override this by --forcemanual flag, you need caution.
-
-+---+
- $ yarn rmadmin -transitionToStandby rm1
- Automatic failover is enabled for org.apache.hadoop.yarn.client.RMHAServiceTarget@1d8299fd
- Refusing to manually manage HA state, since it may cause
- a split-brain scenario or other incorrect state.
- If you are very sure you know what you are doing, please
- specify the forcemanual flag.
-+---+
-
-   See {{{./YarnCommands.html}YarnCommands}} for more details.
-
-** ResourceManager Web UI services
-
-   Assuming a standby RM is up and running, the Standby automatically redirects
-   all web requests to the Active, except for the "About" page.
-
-** Web Services
-
-   Assuming a standby RM is up and running, RM web-services described at
-   {{{./ResourceManagerRest.html}ResourceManager REST APIs}} when invoked on
-   a standby RM are automatically redirected to the Active RM.


[14/50] [abbrv] hadoop git commit: MAPREDUCE-5612. Add javadoc for TaskCompletionEvent.Status. Contributed by Chris Palmer.

Posted by zh...@apache.org.
MAPREDUCE-5612. Add javadoc for TaskCompletionEvent.Status. Contributed by Chris Palmer.


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

Branch: refs/heads/HDFS-7285
Commit: 3d90a92fb331897b845c6639d851d42cb13105f1
Parents: 2827ce2
Author: Akira Ajisaka <aa...@apache.org>
Authored: Thu Feb 26 15:05:17 2015 -0800
Committer: Zhe Zhang <zh...@cloudera.com>
Committed: Mon Mar 2 09:13:52 2015 -0800

----------------------------------------------------------------------
 hadoop-mapreduce-project/CHANGES.txt            |  3 ++
 .../hadoop/mapred/TaskCompletionEvent.java      | 32 +++++++++++++++++++-
 .../hadoop/mapreduce/TaskCompletionEvent.java   | 32 +++++++++++++++++++-
 3 files changed, 65 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/3d90a92f/hadoop-mapreduce-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/CHANGES.txt b/hadoop-mapreduce-project/CHANGES.txt
index 6cb1754..f509d4e 100644
--- a/hadoop-mapreduce-project/CHANGES.txt
+++ b/hadoop-mapreduce-project/CHANGES.txt
@@ -308,6 +308,9 @@ Release 2.7.0 - UNRELEASED
     MAPREDUCE-6264. Remove httpclient dependency from hadoop-mapreduce-client.
     (Brahma Reddy Battula via aajisaka)
 
+    MAPREDUCE-5612. Add javadoc for TaskCompletionEvent.Status.
+    (Chris Palmer via aajisaka)
+
   OPTIMIZATIONS
 
     MAPREDUCE-6169. MergeQueue should release reference to the current item 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3d90a92f/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/TaskCompletionEvent.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/TaskCompletionEvent.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/TaskCompletionEvent.java
index dc4d82e..2bb55a2 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/TaskCompletionEvent.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/TaskCompletionEvent.java
@@ -32,7 +32,37 @@ public class TaskCompletionEvent
     extends org.apache.hadoop.mapreduce.TaskCompletionEvent {
   @InterfaceAudience.Public
   @InterfaceStability.Stable
-  static public enum Status {FAILED, KILLED, SUCCEEDED, OBSOLETE, TIPFAILED};
+  /**
+   *  Task Completion Statuses
+   */
+  static public enum Status {
+    /**
+     * Task Event Attempt failed but there are attempts remaining.
+     */
+    FAILED,
+    /**
+     * Task Event was killed.
+     */
+    KILLED,
+    /**
+     * Task Event was successful.
+     */
+    SUCCEEDED,
+    /**
+     * Used to Override a previously successful event status.
+     * Example:  Map attempt runs and a SUCCEEDED event is sent. Later a task
+     * is retroactively failed due to excessive fetch failure during shuffle
+     * phase. When the retroactive attempt failure occurs, an OBSOLETE event is
+     * sent for the map attempt indicating the prior event is no longer valid.
+     */
+    OBSOLETE,
+    /**
+     * Task Event attempt failed and no further attempts exist.
+     * reached MAX attempts. When a reducer receives a TIPFAILED event it
+     * gives up trying to shuffle data from that map task.
+     */
+    TIPFAILED
+  }
   
   public static final TaskCompletionEvent[] EMPTY_ARRAY = 
 	    new TaskCompletionEvent[0];

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3d90a92f/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/TaskCompletionEvent.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/TaskCompletionEvent.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/TaskCompletionEvent.java
index 31643a9..21c3823 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/TaskCompletionEvent.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/TaskCompletionEvent.java
@@ -36,7 +36,37 @@ import org.apache.hadoop.io.WritableUtils;
 public class TaskCompletionEvent implements Writable{
   @InterfaceAudience.Public
   @InterfaceStability.Evolving
-  static public enum Status {FAILED, KILLED, SUCCEEDED, OBSOLETE, TIPFAILED};
+  /**
+   * Task Completion Statuses
+   */
+  static public enum Status {
+    /**
+     * Task Event Attempt failed but there are attempts remaining.
+     */
+    FAILED,
+    /**
+     * Task Event was killed.
+     */
+    KILLED,
+    /**
+     * Task Event was successful.
+     */
+    SUCCEEDED,
+    /**
+     * Used to Override a previously successful event status.
+     * Example:  Map attempt runs and a SUCCEEDED event is sent. Later a task
+     * is retroactively failed due to excessive fetch failure during shuffle
+     * phase. When the retroactive attempt failure occurs, an OBSOLETE event is
+     * sent for the map attempt indicating the prior event is no longer valid.
+     */
+    OBSOLETE,
+    /**
+     * Task Event attempt failed and no further attempts exist.
+     * reached MAX attempts. When a reducer receives a TIPFAILED event it
+     * gives up trying to shuffle data from that map task.
+     */
+    TIPFAILED
+  }
     
   private int eventId; 
   private String taskTrackerHttp;


[48/50] [abbrv] hadoop git commit: HDFS-7439. Add BlockOpResponseProto's message to the exception messages. Contributed by Takanobu Asanuma

Posted by zh...@apache.org.
HDFS-7439. Add BlockOpResponseProto's message to the exception messages.  Contributed by Takanobu Asanuma


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

Branch: refs/heads/HDFS-7285
Commit: 0e4a23f308c95a99106336f4df7e3e7990dd6ef3
Parents: 72f2c12
Author: Tsz-Wo Nicholas Sze <sz...@hortonworks.com>
Authored: Mon Mar 2 15:03:58 2015 +0800
Committer: Zhe Zhang <zh...@cloudera.com>
Committed: Mon Mar 2 09:13:56 2015 -0800

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |  3 +++
 .../java/org/apache/hadoop/hdfs/DFSClient.java  | 26 ++++++--------------
 .../org/apache/hadoop/hdfs/DFSOutputStream.java | 15 ++++-------
 .../apache/hadoop/hdfs/RemoteBlockReader2.java  | 24 ++++++------------
 .../datatransfer/DataTransferProtoUtil.java     | 26 ++++++++++++++++++++
 .../hadoop/hdfs/server/balancer/Dispatcher.java |  9 +++----
 .../hdfs/server/datanode/DataXceiver.java       | 14 +++--------
 7 files changed, 55 insertions(+), 62 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/0e4a23f3/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 ce35ea2..5ca16af 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -694,6 +694,9 @@ Release 2.7.0 - UNRELEASED
     HDFS-5853. Add "hadoop.user.group.metrics.percentiles.intervals" to
     hdfs-default.xml. (aajisaka)
 
+    HDFS-7439. Add BlockOpResponseProto's message to the exception messages.
+    (Takanobu Asanuma via szetszwo)
+
   OPTIMIZATIONS
 
     HDFS-7454. Reduce memory footprint for AclEntries in NameNode.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0e4a23f3/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
index 792c2dd..abcd847 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
@@ -174,6 +174,7 @@ import org.apache.hadoop.hdfs.protocol.SnapshotAccessControlException;
 import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport;
 import org.apache.hadoop.hdfs.protocol.SnapshottableDirectoryStatus;
 import org.apache.hadoop.hdfs.protocol.UnresolvedPathException;
+import org.apache.hadoop.hdfs.protocol.datatransfer.DataTransferProtoUtil;
 import org.apache.hadoop.hdfs.protocol.datatransfer.IOStreamPair;
 import org.apache.hadoop.hdfs.protocol.datatransfer.Op;
 import org.apache.hadoop.hdfs.protocol.datatransfer.ReplaceDatanodeOnFailure;
@@ -2260,15 +2261,9 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
           final BlockOpResponseProto reply =
             BlockOpResponseProto.parseFrom(PBHelper.vintPrefixed(in));
 
-          if (reply.getStatus() != Status.SUCCESS) {
-            if (reply.getStatus() == Status.ERROR_ACCESS_TOKEN) {
-              throw new InvalidBlockTokenException();
-            } else {
-              throw new IOException("Bad response " + reply + " for block "
-                  + block + " from datanode " + datanodes[j]);
-            }
-          }
-          
+          String logInfo = "for block " + block + " from datanode " + datanodes[j];
+          DataTransferProtoUtil.checkBlockOpStatus(reply, logInfo);
+
           OpBlockChecksumResponseProto checksumData =
             reply.getChecksumResponse();
 
@@ -2425,16 +2420,9 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
           0, 1, true, CachingStrategy.newDefaultStrategy());
       final BlockOpResponseProto reply =
           BlockOpResponseProto.parseFrom(PBHelper.vintPrefixed(in));
-      
-      if (reply.getStatus() != Status.SUCCESS) {
-        if (reply.getStatus() == Status.ERROR_ACCESS_TOKEN) {
-          throw new InvalidBlockTokenException();
-        } else {
-          throw new IOException("Bad response " + reply + " trying to read "
-              + lb.getBlock() + " from datanode " + dn);
-        }
-      }
-      
+      String logInfo = "trying to read " + lb.getBlock() + " from datanode " + dn;
+      DataTransferProtoUtil.checkBlockOpStatus(reply, logInfo);
+
       return PBHelper.convert(reply.getReadOpChecksumInfo().getChecksum().getType());
     } finally {
       IOUtils.cleanup(null, pair.in, pair.out);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0e4a23f3/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java
index b3e8c97..dc2f674 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java
@@ -69,6 +69,7 @@ import org.apache.hadoop.hdfs.protocol.SnapshotAccessControlException;
 import org.apache.hadoop.hdfs.protocol.UnresolvedPathException;
 import org.apache.hadoop.hdfs.protocol.datatransfer.BlockConstructionStage;
 import org.apache.hadoop.hdfs.protocol.datatransfer.DataTransferProtocol;
+import org.apache.hadoop.hdfs.protocol.datatransfer.DataTransferProtoUtil;
 import org.apache.hadoop.hdfs.protocol.datatransfer.IOStreamPair;
 import org.apache.hadoop.hdfs.protocol.datatransfer.InvalidEncryptionKeyException;
 import org.apache.hadoop.hdfs.protocol.datatransfer.PacketHeader;
@@ -1469,16 +1470,10 @@ public class DFSOutputStream extends FSOutputSummer
             checkRestart = true;
             throw new IOException("A datanode is restarting.");
           }
-          if (pipelineStatus != SUCCESS) {
-            if (pipelineStatus == Status.ERROR_ACCESS_TOKEN) {
-              throw new InvalidBlockTokenException(
-                  "Got access token error for connect ack with firstBadLink as "
-                      + firstBadLink);
-            } else {
-              throw new IOException("Bad connect ack with firstBadLink as "
-                  + firstBadLink);
-            }
-          }
+
+          String logInfo = "ack with firstBadLink as " + firstBadLink;
+          DataTransferProtoUtil.checkBlockOpStatus(resp, logInfo);
+
           assert null == blockStream : "Previous blockStream unclosed";
           blockStream = out;
           result =  true; // success

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0e4a23f3/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader2.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader2.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader2.java
index 3f133b6..9245a84 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader2.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader2.java
@@ -45,7 +45,6 @@ import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ReadOpChecksumIn
 import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status;
 import org.apache.hadoop.hdfs.protocolPB.PBHelper;
 import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
-import org.apache.hadoop.hdfs.security.token.block.InvalidBlockTokenException;
 import org.apache.hadoop.hdfs.server.datanode.CachingStrategy;
 import org.apache.hadoop.hdfs.shortcircuit.ClientMmap;
 import org.apache.hadoop.net.NetUtils;
@@ -448,22 +447,13 @@ public class RemoteBlockReader2  implements BlockReader {
       BlockOpResponseProto status, Peer peer,
       ExtendedBlock block, String file)
       throws IOException {
-    if (status.getStatus() != Status.SUCCESS) {
-      if (status.getStatus() == Status.ERROR_ACCESS_TOKEN) {
-        throw new InvalidBlockTokenException(
-            "Got access token error for OP_READ_BLOCK, self="
-                + peer.getLocalAddressString() + ", remote="
-                + peer.getRemoteAddressString() + ", for file " + file
-                + ", for pool " + block.getBlockPoolId() + " block " 
-                + block.getBlockId() + "_" + block.getGenerationStamp());
-      } else {
-        throw new IOException("Got error for OP_READ_BLOCK, self="
-            + peer.getLocalAddressString() + ", remote="
-            + peer.getRemoteAddressString() + ", for file " + file
-            + ", for pool " + block.getBlockPoolId() + " block " 
-            + block.getBlockId() + "_" + block.getGenerationStamp());
-      }
-    }
+    String logInfo = "for OP_READ_BLOCK"
+      + ", self=" + peer.getLocalAddressString()
+      + ", remote=" + peer.getRemoteAddressString()
+      + ", for file " + file
+      + ", for pool " + block.getBlockPoolId()
+      + " block " + block.getBlockId() + "_" + block.getGenerationStamp();
+    DataTransferProtoUtil.checkBlockOpStatus(status, logInfo);
   }
   
   @Override

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0e4a23f3/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/DataTransferProtoUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/DataTransferProtoUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/DataTransferProtoUtil.java
index 2ef3c3f..284281a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/DataTransferProtoUtil.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/DataTransferProtoUtil.java
@@ -17,11 +17,16 @@
  */
 package org.apache.hadoop.hdfs.protocol.datatransfer;
 
+import java.io.IOException;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hdfs.net.Peer;
+import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BaseHeaderProto;
+import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BlockOpResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status;
 import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ChecksumProto;
 import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ClientOperationHeaderProto;
 import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpWriteBlockProto;
@@ -29,6 +34,7 @@ import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.DataTransferTrac
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ChecksumTypeProto;
 import org.apache.hadoop.hdfs.protocolPB.PBHelper;
 import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
+import org.apache.hadoop.hdfs.security.token.block.InvalidBlockTokenException;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.util.DataChecksum;
 import org.apache.htrace.Span;
@@ -119,4 +125,24 @@ public abstract class DataTransferProtoUtil {
     }
     return scope;
   }
+
+  public static void checkBlockOpStatus(
+          BlockOpResponseProto response,
+          String logInfo) throws IOException {
+    if (response.getStatus() != Status.SUCCESS) {
+      if (response.getStatus() == Status.ERROR_ACCESS_TOKEN) {
+        throw new InvalidBlockTokenException(
+          "Got access token error"
+          + ", status message " + response.getMessage()
+          + ", " + logInfo
+        );
+      } else {
+        throw new IOException(
+          "Got error"
+          + ", status message " + response.getMessage()
+          + ", " + logInfo
+        );
+      }
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0e4a23f3/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/Dispatcher.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/Dispatcher.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/Dispatcher.java
index fa17cac..a3fd251 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/Dispatcher.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/Dispatcher.java
@@ -53,6 +53,7 @@ import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+import org.apache.hadoop.hdfs.protocol.datatransfer.DataTransferProtoUtil;
 import org.apache.hadoop.hdfs.protocol.datatransfer.IOStreamPair;
 import org.apache.hadoop.hdfs.protocol.datatransfer.Sender;
 import org.apache.hadoop.hdfs.protocol.datatransfer.TrustedChannelResolver;
@@ -357,12 +358,8 @@ public class Dispatcher {
         // read intermediate responses
         response = BlockOpResponseProto.parseFrom(vintPrefixed(in));
       }
-      if (response.getStatus() != Status.SUCCESS) {
-        if (response.getStatus() == Status.ERROR_ACCESS_TOKEN) {
-          throw new IOException("block move failed due to access token error");
-        }
-        throw new IOException("block move is failed: " + response.getMessage());
-      }
+      String logInfo = "block move is failed";
+      DataTransferProtoUtil.checkBlockOpStatus(response, logInfo);
     }
 
     /** reset the object */

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0e4a23f3/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataXceiver.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataXceiver.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataXceiver.java
index 6a2250f..e9547a8 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataXceiver.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataXceiver.java
@@ -1116,16 +1116,10 @@ class DataXceiver extends Receiver implements Runnable {
         BlockOpResponseProto copyResponse = BlockOpResponseProto.parseFrom(
             PBHelper.vintPrefixed(proxyReply));
         
-        if (copyResponse.getStatus() != SUCCESS) {
-          if (copyResponse.getStatus() == ERROR_ACCESS_TOKEN) {
-            throw new IOException("Copy block " + block + " from "
-                + proxySock.getRemoteSocketAddress()
-                + " failed due to access token error");
-          }
-          throw new IOException("Copy block " + block + " from "
-              + proxySock.getRemoteSocketAddress() + " failed");
-        }
-        
+        String logInfo = "copy block " + block + " from "
+            + proxySock.getRemoteSocketAddress();
+        DataTransferProtoUtil.checkBlockOpStatus(copyResponse, logInfo);
+
         // get checksum info about the block we're copying
         ReadOpChecksumInfoProto checksumInfo = copyResponse.getReadOpChecksumInfo();
         DataChecksum remoteChecksum = DataTransferProtoUtil.fromProto(


[02/50] [abbrv] hadoop git commit: HDFS-7495. Remove updatePosition argument from DFSInputStream#getBlockAt() (cmccabe)

Posted by zh...@apache.org.
HDFS-7495. Remove updatePosition argument from DFSInputStream#getBlockAt() (cmccabe)


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

Branch: refs/heads/HDFS-7285
Commit: 11a1c72ceb010b8048db79417ad65646047f9111
Parents: 8d184d1
Author: Colin Patrick Mccabe <cm...@cloudera.com>
Authored: Wed Feb 25 13:29:31 2015 -0800
Committer: Zhe Zhang <zh...@cloudera.com>
Committed: Mon Mar 2 09:13:50 2015 -0800

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |  3 ++
 .../org/apache/hadoop/hdfs/DFSInputStream.java  | 33 ++++++++------------
 2 files changed, 16 insertions(+), 20 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/11a1c72c/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 4ca10da..e0f9267 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -668,6 +668,9 @@ Release 2.7.0 - UNRELEASED
 
     HDFS-7740. Test truncate with DataNodes restarting. (yliu)
 
+    HDFS-7495. Remove updatePosition argument from DFSInputStream#getBlockAt()
+    (cmccabe)
+
   OPTIMIZATIONS
 
     HDFS-7454. Reduce memory footprint for AclEntries in NameNode.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/11a1c72c/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java
index c408524..cf8015f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java
@@ -434,12 +434,10 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
    * Fetch it from the namenode if not cached.
    * 
    * @param offset block corresponding to this offset in file is returned
-   * @param updatePosition whether to update current position
    * @return located block
    * @throws IOException
    */
-  private LocatedBlock getBlockAt(long offset,
-      boolean updatePosition) throws IOException {
+  private LocatedBlock getBlockAt(long offset) throws IOException {
     synchronized(infoLock) {
       assert (locatedBlocks != null) : "locatedBlocks is null";
 
@@ -449,7 +447,6 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
       if (offset < 0 || offset >= getFileLength()) {
         throw new IOException("offset < 0 || offset >= getFileLength(), offset="
             + offset
-            + ", updatePosition=" + updatePosition
             + ", locatedBlocks=" + locatedBlocks);
       }
       else if (offset >= locatedBlocks.getFileLength()) {
@@ -470,17 +467,6 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
         }
         blk = locatedBlocks.get(targetBlockIdx);
       }
-
-      // update current position
-      if (updatePosition) {
-        // synchronized not strictly needed, since we only get here
-        // from synchronized caller methods
-        synchronized(this) {
-          pos = offset;
-          blockEnd = blk.getStartOffset() + blk.getBlockSize() - 1;
-          currentLocatedBlock = blk;
-        }
-      }
       return blk;
     }
   }
@@ -604,7 +590,14 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
       //
       // Compute desired block
       //
-      LocatedBlock targetBlock = getBlockAt(target, true);
+      LocatedBlock targetBlock = getBlockAt(target);
+
+      // update current position
+      this.pos = target;
+      this.blockEnd = targetBlock.getStartOffset() +
+            targetBlock.getBlockSize() - 1;
+      this.currentLocatedBlock = targetBlock;
+
       assert (target==pos) : "Wrong postion " + pos + " expect " + target;
       long offsetIntoBlock = target - targetBlock.getStartOffset();
 
@@ -979,7 +972,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
         }
         deadNodes.clear(); //2nd option is to remove only nodes[blockId]
         openInfo();
-        block = getBlockAt(block.getStartOffset(), false);
+        block = getBlockAt(block.getStartOffset());
         failures++;
         continue;
       }
@@ -1056,7 +1049,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
       byte[] buf, int offset,
       Map<ExtendedBlock, Set<DatanodeInfo>> corruptedBlockMap)
       throws IOException {
-    block = getBlockAt(block.getStartOffset(), false);
+    block = getBlockAt(block.getStartOffset());
     while (true) {
       DNAddrPair addressPair = chooseDataNode(block, null);
       try {
@@ -1108,7 +1101,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
       // start of the loop.
       CachingStrategy curCachingStrategy;
       boolean allowShortCircuitLocalReads;
-      block = getBlockAt(block.getStartOffset(), false);
+      block = getBlockAt(block.getStartOffset());
       synchronized(infoLock) {
         curCachingStrategy = cachingStrategy;
         allowShortCircuitLocalReads = !shortCircuitForbidden();
@@ -1208,7 +1201,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
     ByteBuffer bb = null;
     int len = (int) (end - start + 1);
     int hedgedReadId = 0;
-    block = getBlockAt(block.getStartOffset(), false);
+    block = getBlockAt(block.getStartOffset());
     while (true) {
       // see HDFS-6591, this metric is used to verify/catch unnecessary loops
       hedgedReadOpsLoopNumForTesting++;


[26/50] [abbrv] hadoop git commit: HDFS-6753. Initialize checkDisk when DirectoryScanner not able to get files list for scanning (Contributed by J.Andreina)

Posted by zh...@apache.org.
HDFS-6753. Initialize checkDisk when DirectoryScanner not able to get files list for scanning (Contributed by J.Andreina)


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

Branch: refs/heads/HDFS-7285
Commit: 54bcb5f7e3fa8c1e4f3f325a4c3b6a24f61e4687
Parents: 04f0dae
Author: Vinayakumar B <vi...@apache.org>
Authored: Fri Feb 27 16:36:28 2015 +0530
Committer: Zhe Zhang <zh...@cloudera.com>
Committed: Mon Mar 2 09:13:53 2015 -0800

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt             |  3 +++
 .../apache/hadoop/hdfs/server/datanode/DataNode.java    |  2 +-
 .../hadoop/hdfs/server/datanode/DirectoryScanner.java   | 12 +++++++++---
 .../hdfs/server/datanode/TestDirectoryScanner.java      |  9 ++++++---
 4 files changed, 19 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/54bcb5f7/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 ba553dc..8556afd 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -1040,6 +1040,9 @@ Release 2.7.0 - UNRELEASED
     HDFS-7774. Unresolved symbols error while compiling HDFS on Windows 7/32 bit.
     (Kiran Kumar M R via cnauroth)
 
+    HDFS-6753. Initialize checkDisk when DirectoryScanner not able to get
+    files list for scanning (J.Andreina via vinayakumarb)
+
     BREAKDOWN OF HDFS-7584 SUBTASKS AND RELATED JIRAS
 
       HDFS-7720. Quota by Storage Type API, tools and ClientNameNode

http://git-wip-us.apache.org/repos/asf/hadoop/blob/54bcb5f7/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
index f233e02..92ddb7b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
@@ -815,7 +815,7 @@ public class DataNode extends ReconfigurableBase
       reason = "verifcation is not supported by SimulatedFSDataset";
     } 
     if (reason == null) {
-      directoryScanner = new DirectoryScanner(data, conf);
+      directoryScanner = new DirectoryScanner(this, data, conf);
       directoryScanner.start();
     } else {
       LOG.info("Periodic Directory Tree Verification scan is disabled because " +

http://git-wip-us.apache.org/repos/asf/hadoop/blob/54bcb5f7/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DirectoryScanner.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DirectoryScanner.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DirectoryScanner.java
index 09c2914..c7ee21e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DirectoryScanner.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DirectoryScanner.java
@@ -63,6 +63,7 @@ public class DirectoryScanner implements Runnable {
   private final long scanPeriodMsecs;
   private volatile boolean shouldRun = false;
   private boolean retainDiffs = false;
+  private final DataNode datanode;
 
   final ScanInfoPerBlockPool diffs = new ScanInfoPerBlockPool();
   final Map<String, Stats> stats = new HashMap<String, Stats>();
@@ -308,7 +309,8 @@ public class DirectoryScanner implements Runnable {
     }
   }
 
-  DirectoryScanner(FsDatasetSpi<?> dataset, Configuration conf) {
+  DirectoryScanner(DataNode datanode, FsDatasetSpi<?> dataset, Configuration conf) {
+    this.datanode = datanode;
     this.dataset = dataset;
     int interval = conf.getInt(DFSConfigKeys.DFS_DATANODE_DIRECTORYSCAN_INTERVAL_KEY,
         DFSConfigKeys.DFS_DATANODE_DIRECTORYSCAN_INTERVAL_DEFAULT);
@@ -547,7 +549,7 @@ public class DirectoryScanner implements Runnable {
     for (int i = 0; i < volumes.size(); i++) {
       if (isValid(dataset, volumes.get(i))) {
         ReportCompiler reportCompiler =
-          new ReportCompiler(volumes.get(i));
+          new ReportCompiler(datanode,volumes.get(i));
         Future<ScanInfoPerBlockPool> result = 
           reportCompileThreadPool.submit(reportCompiler);
         compilersInProgress.put(i, result);
@@ -585,8 +587,10 @@ public class DirectoryScanner implements Runnable {
   private static class ReportCompiler 
   implements Callable<ScanInfoPerBlockPool> {
     private final FsVolumeSpi volume;
+    private final DataNode datanode;
 
-    public ReportCompiler(FsVolumeSpi volume) {
+    public ReportCompiler(DataNode datanode, FsVolumeSpi volume) {
+      this.datanode = datanode;
       this.volume = volume;
     }
 
@@ -611,6 +615,8 @@ public class DirectoryScanner implements Runnable {
         files = FileUtil.listFiles(dir);
       } catch (IOException ioe) {
         LOG.warn("Exception occured while compiling report: ", ioe);
+        // Initiate a check on disk failure.
+        datanode.checkDiskErrorAsync();
         // Ignore this directory and proceed.
         return report;
       }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/54bcb5f7/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDirectoryScanner.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDirectoryScanner.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDirectoryScanner.java
index 681768e..0e0e959 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDirectoryScanner.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDirectoryScanner.java
@@ -299,10 +299,11 @@ public class TestDirectoryScanner {
         .build();
     try {
       cluster.waitActive();
+      DataNode dataNode = cluster.getDataNodes().get(0);
       bpid = cluster.getNamesystem().getBlockPoolId();
       fds = DataNodeTestUtils.getFSDataset(cluster.getDataNodes().get(0));
       client = cluster.getFileSystem().getClient();
-      scanner = new DirectoryScanner(fds, CONF);
+      scanner = new DirectoryScanner(dataNode, fds, CONF);
       scanner.setRetainDiffs(true);
       FsDatasetTestUtil.stopLazyWriter(cluster.getDataNodes().get(0));
 
@@ -340,9 +341,10 @@ public class TestDirectoryScanner {
     try {
       cluster.waitActive();
       bpid = cluster.getNamesystem().getBlockPoolId();
+      DataNode dataNode = cluster.getDataNodes().get(0);
       fds = DataNodeTestUtils.getFSDataset(cluster.getDataNodes().get(0));
       client = cluster.getFileSystem().getClient();
-      scanner = new DirectoryScanner(fds, CONF);
+      scanner = new DirectoryScanner(dataNode, fds, CONF);
       scanner.setRetainDiffs(true);
       FsDatasetTestUtil.stopLazyWriter(cluster.getDataNodes().get(0));
 
@@ -389,7 +391,8 @@ public class TestDirectoryScanner {
       client = cluster.getFileSystem().getClient();
       CONF.setInt(DFSConfigKeys.DFS_DATANODE_DIRECTORYSCAN_THREADS_KEY,
                   parallelism);
-      scanner = new DirectoryScanner(fds, CONF);
+      DataNode dataNode = cluster.getDataNodes().get(0);
+      scanner = new DirectoryScanner(dataNode, fds, CONF);
       scanner.setRetainDiffs(true);
 
       // Add files with 100 blocks


[40/50] [abbrv] hadoop git commit: MAPREDUCE-5653. DistCp does not honour config-overrides for mapreduce.[map, reduce].memory.mb (Ratandeep Ratti via aw)

Posted by zh...@apache.org.
MAPREDUCE-5653. DistCp does not honour config-overrides for mapreduce.[map,reduce].memory.mb (Ratandeep Ratti via aw)


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

Branch: refs/heads/HDFS-7285
Commit: d6c1b627d30bc18fedbbb5c0fa4eeeeba98e6043
Parents: 7797249
Author: Allen Wittenauer <aw...@apache.org>
Authored: Sat Feb 28 22:53:38 2015 -0800
Committer: Zhe Zhang <zh...@cloudera.com>
Committed: Mon Mar 2 09:13:54 2015 -0800

----------------------------------------------------------------------
 hadoop-mapreduce-project/CHANGES.txt                      |  3 +++
 .../hadoop-distcp/src/main/resources/distcp-default.xml   | 10 ----------
 2 files changed, 3 insertions(+), 10 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/d6c1b627/hadoop-mapreduce-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/CHANGES.txt b/hadoop-mapreduce-project/CHANGES.txt
index f509d4e..ccd24a6 100644
--- a/hadoop-mapreduce-project/CHANGES.txt
+++ b/hadoop-mapreduce-project/CHANGES.txt
@@ -6,6 +6,9 @@ Trunk (Unreleased)
     MAPREDUCE-5785. Derive heap size or mapreduce.*.memory.mb automatically.
     (Gera Shegalov and Karthik Kambatla via gera)
 
+    MAPREDUCE-5653. DistCp does not honour config-overrides for
+    mapreduce.[map,reduce].memory.mb (Ratandeep Ratti via aw)
+
   NEW FEATURES
 
     MAPREDUCE-778. Rumen Anonymizer. (Amar Kamat and Chris Douglas via amarrk)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d6c1b627/hadoop-tools/hadoop-distcp/src/main/resources/distcp-default.xml
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-distcp/src/main/resources/distcp-default.xml b/hadoop-tools/hadoop-distcp/src/main/resources/distcp-default.xml
index f50dddd..6e1154e 100644
--- a/hadoop-tools/hadoop-distcp/src/main/resources/distcp-default.xml
+++ b/hadoop-tools/hadoop-distcp/src/main/resources/distcp-default.xml
@@ -32,16 +32,6 @@
     </property>
 
     <property>
-        <name>mapred.job.map.memory.mb</name>
-        <value>1024</value>
-    </property>
-
-    <property>
-        <name>mapred.job.reduce.memory.mb</name>
-        <value>1024</value>
-    </property>
-
-    <property>
         <name>mapred.reducer.new-api</name>
         <value>true</value>
     </property>


[49/50] [abbrv] hadoop git commit: HADOOP-11658. Externalize io.compression.codecs property. Contributed by Kai Zheng.

Posted by zh...@apache.org.
HADOOP-11658. Externalize io.compression.codecs property. Contributed by Kai Zheng.


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

Branch: refs/heads/HDFS-7285
Commit: 22e6b2d3ea7a0abb630324e74a74abb533a96ef0
Parents: 7deb79f
Author: Akira Ajisaka <aa...@apache.org>
Authored: Mon Mar 2 01:09:54 2015 -0800
Committer: Zhe Zhang <zh...@cloudera.com>
Committed: Mon Mar 2 09:13:56 2015 -0800

----------------------------------------------------------------------
 hadoop-common-project/hadoop-common/CHANGES.txt |  3 +++
 .../hadoop/fs/CommonConfigurationKeys.java      | 17 +++++++++++-----
 .../io/compress/CompressionCodecFactory.java    | 21 +++++++++++++-------
 .../hadoop/io/compress/TestCodecFactory.java    |  3 ++-
 4 files changed, 31 insertions(+), 13 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/22e6b2d3/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 4c0c375..b8ed286 100644
--- a/hadoop-common-project/hadoop-common/CHANGES.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES.txt
@@ -642,6 +642,9 @@ Release 2.7.0 - UNRELEASED
     HADOOP-10976. moving the source code of hadoop-tools docs to the
     directory under hadoop-tools (Masatake Iwasaki via aw)
 
+    HADOOP-11658. Externalize io.compression.codecs property.
+    (Kai Zheng via aajisaka)
+
   OPTIMIZATIONS
 
     HADOOP-11323. WritableComparator#compare keeps reference to byte array.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/22e6b2d3/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeys.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeys.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeys.java
index 442dc7d..7575496 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeys.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeys.java
@@ -91,17 +91,24 @@ public class CommonConfigurationKeys extends CommonConfigurationKeysPublic {
   public static final String IPC_CALLQUEUE_IMPL_KEY = "callqueue.impl";
   public static final String IPC_CALLQUEUE_IDENTITY_PROVIDER_KEY = "identity-provider.impl";
 
+  /** This is for specifying the implementation for the mappings from
+   * hostnames to the racks they belong to
+   */
+  public static final String  NET_TOPOLOGY_CONFIGURED_NODE_MAPPING_KEY =
+      "net.topology.configured.node.mapping";
+
+  /**
+   * Supported compression codec classes
+   */
+  public static final String IO_COMPRESSION_CODECS_KEY = "io.compression.codecs";
+
   /** Internal buffer size for Lzo compressor/decompressors */
   public static final String  IO_COMPRESSION_CODEC_LZO_BUFFERSIZE_KEY =
     "io.compression.codec.lzo.buffersize";
+
   /** Default value for IO_COMPRESSION_CODEC_LZO_BUFFERSIZE_KEY */
   public static final int     IO_COMPRESSION_CODEC_LZO_BUFFERSIZE_DEFAULT =
     64*1024;
-  /** This is for specifying the implementation for the mappings from
-   * hostnames to the racks they belong to
-   */
-  public static final String  NET_TOPOLOGY_CONFIGURED_NODE_MAPPING_KEY =
-    "net.topology.configured.node.mapping";
 
   /** Internal buffer size for Snappy compressor/decompressors */
   public static final String IO_COMPRESSION_CODEC_SNAPPY_BUFFERSIZE_KEY =

http://git-wip-us.apache.org/repos/asf/hadoop/blob/22e6b2d3/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/CompressionCodecFactory.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/CompressionCodecFactory.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/CompressionCodecFactory.java
index eb35759..7476a15 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/CompressionCodecFactory.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/CompressionCodecFactory.java
@@ -24,6 +24,7 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.util.ReflectionUtils;
 
@@ -106,7 +107,8 @@ public class CompressionCodecFactory {
    * @param conf the configuration to look in
    * @return a list of the {@link CompressionCodec} classes
    */
-  public static List<Class<? extends CompressionCodec>> getCodecClasses(Configuration conf) {
+  public static List<Class<? extends CompressionCodec>> getCodecClasses(
+      Configuration conf) {
     List<Class<? extends CompressionCodec>> result
       = new ArrayList<Class<? extends CompressionCodec>>();
     // Add codec classes discovered via service loading
@@ -118,7 +120,8 @@ public class CompressionCodecFactory {
       }
     }
     // Add codec classes from configuration
-    String codecsString = conf.get("io.compression.codecs");
+    String codecsString = conf.get(
+        CommonConfigurationKeys.IO_COMPRESSION_CODECS_KEY);
     if (codecsString != null) {
       StringTokenizer codecSplit = new StringTokenizer(codecsString, ",");
       while (codecSplit.hasMoreElements()) {
@@ -161,7 +164,7 @@ public class CompressionCodecFactory {
         buf.append(itr.next().getName());
       }
     }
-    conf.set("io.compression.codecs", buf.toString());   
+    conf.set(CommonConfigurationKeys.IO_COMPRESSION_CODECS_KEY, buf.toString());
   }
   
   /**
@@ -172,7 +175,8 @@ public class CompressionCodecFactory {
     codecs = new TreeMap<String, CompressionCodec>();
     codecsByClassName = new HashMap<String, CompressionCodec>();
     codecsByName = new HashMap<String, CompressionCodec>();
-    List<Class<? extends CompressionCodec>> codecClasses = getCodecClasses(conf);
+    List<Class<? extends CompressionCodec>> codecClasses =
+        getCodecClasses(conf);
     if (codecClasses == null || codecClasses.isEmpty()) {
       addCodec(new GzipCodec());
       addCodec(new DefaultCodec());      
@@ -193,7 +197,8 @@ public class CompressionCodecFactory {
     CompressionCodec result = null;
     if (codecs != null) {
       String filename = file.getName();
-      String reversedFilename = new StringBuilder(filename).reverse().toString();
+      String reversedFilename =
+          new StringBuilder(filename).reverse().toString();
       SortedMap<String, CompressionCodec> subMap = 
         codecs.headMap(reversedFilename);
       if (!subMap.isEmpty()) {
@@ -239,7 +244,8 @@ public class CompressionCodecFactory {
       }
       CompressionCodec codec = getCodecByClassName(codecName);
       if (codec == null) {
-        // trying to get the codec by name in case the name was specified instead a class
+        // trying to get the codec by name in case the name was specified
+        // instead a class
         codec = codecsByName.get(codecName.toLowerCase());
       }
       return codec;
@@ -260,7 +266,8 @@ public class CompressionCodecFactory {
      * @param codecName the canonical class name of the codec
      * @return the codec class
      */
-    public Class<? extends CompressionCodec> getCodecClassByName(String codecName) {
+    public Class<? extends CompressionCodec> getCodecClassByName(
+        String codecName) {
       CompressionCodec codec = getCodecByName(codecName);
       if (codec == null) {
         return null;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/22e6b2d3/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/compress/TestCodecFactory.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/compress/TestCodecFactory.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/compress/TestCodecFactory.java
index 7601211..3b81a3f 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/compress/TestCodecFactory.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/compress/TestCodecFactory.java
@@ -24,6 +24,7 @@ import java.io.OutputStream;
 import java.util.*;
 
 import junit.framework.TestCase;
+import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.conf.Configuration;
 
@@ -258,7 +259,7 @@ public class TestCodecFactory extends TestCase {
     checkCodec("overridden factory for gzip codec", NewGzipCodec.class, codec);
     
     Configuration conf = new Configuration();
-    conf.set("io.compression.codecs", 
+    conf.set(CommonConfigurationKeys.IO_COMPRESSION_CODECS_KEY,
         "   org.apache.hadoop.io.compress.GzipCodec   , " +
         "    org.apache.hadoop.io.compress.DefaultCodec  , " +
         " org.apache.hadoop.io.compress.BZip2Codec   ");


[17/50] [abbrv] hadoop git commit: YARN-3217. Remove httpclient dependency from hadoop-yarn-server-web-proxy. Contributed by Brahma Reddy Battula.

Posted by zh...@apache.org.
YARN-3217. Remove httpclient dependency from hadoop-yarn-server-web-proxy. Contributed by Brahma Reddy Battula.


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

Branch: refs/heads/HDFS-7285
Commit: 41d995245ea2c28b68a96ba60dde79279721348c
Parents: 982165c
Author: Tsuyoshi Ozawa <oz...@apache.org>
Authored: Fri Feb 27 00:22:46 2015 +0900
Committer: Zhe Zhang <zh...@cloudera.com>
Committed: Mon Mar 2 09:13:52 2015 -0800

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                 |  3 ++
 .../hadoop-yarn-server-web-proxy/pom.xml        |  4 --
 .../server/webproxy/WebAppProxyServlet.java     | 46 ++++++++++----------
 3 files changed, 26 insertions(+), 27 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/41d99524/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index ac3cbb2..a635592 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -327,6 +327,9 @@ Release 2.7.0 - UNRELEASED
     YARN-2797. Add -help to yarn logs and nodes CLI command. 
     (Jagadesh Kiran N via devaraj)
 
+    YARN-3217. Remove httpclient dependency from hadoop-yarn-server-web-proxy.
+    (Brahma Reddy Battula via ozawa).
+
   OPTIMIZATIONS
 
     YARN-2990. FairScheduler's delay-scheduling always waits for node-local and 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/41d99524/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-web-proxy/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-web-proxy/pom.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-web-proxy/pom.xml
index fdba1fe..9801064 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-web-proxy/pom.xml
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-web-proxy/pom.xml
@@ -79,10 +79,6 @@
     </dependency>
 
     <dependency>
-      <groupId>commons-httpclient</groupId>
-      <artifactId>commons-httpclient</artifactId>
-    </dependency>
-    <dependency>
       <groupId>com.google.guava</groupId>
       <artifactId>guava</artifactId>
     </dependency>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/41d99524/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-web-proxy/src/main/java/org/apache/hadoop/yarn/server/webproxy/WebAppProxyServlet.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-web-proxy/src/main/java/org/apache/hadoop/yarn/server/webproxy/WebAppProxyServlet.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-web-proxy/src/main/java/org/apache/hadoop/yarn/server/webproxy/WebAppProxyServlet.java
index 47f7769..fd98c80 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-web-proxy/src/main/java/org/apache/hadoop/yarn/server/webproxy/WebAppProxyServlet.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-web-proxy/src/main/java/org/apache/hadoop/yarn/server/webproxy/WebAppProxyServlet.java
@@ -40,13 +40,6 @@ import javax.servlet.http.HttpServletRequest;
 import javax.servlet.http.HttpServletResponse;
 import javax.ws.rs.core.UriBuilder;
 
-import org.apache.commons.httpclient.Header;
-import org.apache.commons.httpclient.HostConfiguration;
-import org.apache.commons.httpclient.HttpClient;
-import org.apache.commons.httpclient.HttpMethod;
-import org.apache.commons.httpclient.cookie.CookiePolicy;
-import org.apache.commons.httpclient.methods.GetMethod;
-import org.apache.commons.httpclient.params.HttpClientParams;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ApplicationReport;
@@ -59,8 +52,15 @@ import org.apache.hadoop.yarn.util.TrackingUriPlugin;
 import org.apache.hadoop.yarn.webapp.MimeType;
 import org.apache.hadoop.yarn.webapp.hamlet.Hamlet;
 import org.apache.hadoop.yarn.webapp.util.WebAppUtils;
+import org.apache.http.Header;
+import org.apache.http.HttpResponse;
 import org.apache.http.NameValuePair;
+import org.apache.http.client.methods.HttpGet;
+import org.apache.http.client.params.ClientPNames;
+import org.apache.http.client.params.CookiePolicy;
 import org.apache.http.client.utils.URLEncodedUtils;
+import org.apache.http.conn.params.ConnRoutePNames;
+import org.apache.http.impl.client.DefaultHttpClient;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -155,23 +155,22 @@ public class WebAppProxyServlet extends HttpServlet {
   private static void proxyLink(HttpServletRequest req, 
       HttpServletResponse resp, URI link, Cookie c, String proxyHost)
       throws IOException {
-    org.apache.commons.httpclient.URI uri = 
-      new org.apache.commons.httpclient.URI(link.toString(), false);
-    HttpClientParams params = new HttpClientParams();
-    params.setCookiePolicy(CookiePolicy.BROWSER_COMPATIBILITY);
-    params.setBooleanParameter(HttpClientParams.ALLOW_CIRCULAR_REDIRECTS, true);
-    HttpClient client = new HttpClient(params);
+    DefaultHttpClient client = new DefaultHttpClient();
+    client
+        .getParams()
+        .setParameter(ClientPNames.COOKIE_POLICY,
+            CookiePolicy.BROWSER_COMPATIBILITY)
+        .setBooleanParameter(ClientPNames.ALLOW_CIRCULAR_REDIRECTS, true);
     // Make sure we send the request from the proxy address in the config
     // since that is what the AM filter checks against. IP aliasing or
     // similar could cause issues otherwise.
-    HostConfiguration config = new HostConfiguration();
     InetAddress localAddress = InetAddress.getByName(proxyHost);
     if (LOG.isDebugEnabled()) {
       LOG.debug("local InetAddress for proxy host: {}", localAddress);
     }
-    config.setLocalAddress(localAddress);
-    HttpMethod method = new GetMethod(uri.getEscapedURI());
-    method.setRequestHeader("Connection","close");
+    client.getParams()
+        .setParameter(ConnRoutePNames.LOCAL_ADDRESS, localAddress);
+    HttpGet httpGet = new HttpGet(link);
     @SuppressWarnings("unchecked")
     Enumeration<String> names = req.getHeaderNames();
     while(names.hasMoreElements()) {
@@ -181,30 +180,31 @@ public class WebAppProxyServlet extends HttpServlet {
         if (LOG.isDebugEnabled()) {
           LOG.debug("REQ HEADER: {} : {}", name, value);
         }
-        method.setRequestHeader(name, value);
+        httpGet.setHeader(name, value);
       }
     }
 
     String user = req.getRemoteUser();
     if (user != null && !user.isEmpty()) {
-      method.setRequestHeader("Cookie",
+      httpGet.setHeader("Cookie",
           PROXY_USER_COOKIE_NAME + "=" + URLEncoder.encode(user, "ASCII"));
     }
     OutputStream out = resp.getOutputStream();
     try {
-      resp.setStatus(client.executeMethod(config, method));
-      for(Header header : method.getResponseHeaders()) {
+      HttpResponse httpResp = client.execute(httpGet);
+      resp.setStatus(httpResp.getStatusLine().getStatusCode());
+      for (Header header : httpResp.getAllHeaders()) {
         resp.setHeader(header.getName(), header.getValue());
       }
       if (c != null) {
         resp.addCookie(c);
       }
-      InputStream in = method.getResponseBodyAsStream();
+      InputStream in = httpResp.getEntity().getContent();
       if (in != null) {
         IOUtils.copyBytes(in, out, 4096, true);
       }
     } finally {
-      method.releaseConnection();
+      httpGet.releaseConnection();
     }
   }
   


[03/50] [abbrv] hadoop git commit: HADOOP-11632. Cleanup Find.java to remove SupressWarnings annotations. Contributed by Akira AJISAKA.

Posted by zh...@apache.org.
HADOOP-11632. Cleanup Find.java to remove SupressWarnings annotations. Contributed by Akira AJISAKA.


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

Branch: refs/heads/HDFS-7285
Commit: afc8188847a702d7e256b285555efa63fa47a2a8
Parents: cadfb71
Author: Tsuyoshi Ozawa <oz...@apache.org>
Authored: Wed Feb 25 16:25:04 2015 +0900
Committer: Zhe Zhang <zh...@cloudera.com>
Committed: Mon Mar 2 09:13:50 2015 -0800

----------------------------------------------------------------------
 hadoop-common-project/hadoop-common/CHANGES.txt |  3 ++
 .../org/apache/hadoop/fs/shell/find/Find.java   | 29 ++++++++++++--------
 2 files changed, 20 insertions(+), 12 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/afc81888/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 a5a11b9..988eed0 100644
--- a/hadoop-common-project/hadoop-common/CHANGES.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES.txt
@@ -628,6 +628,9 @@ Release 2.7.0 - UNRELEASED
     HADOOP-11495. Convert site documentation from apt to markdown
     (Masatake Iwasaki via aw)
 
+    HADOOP-11632. Cleanup Find.java to remove SupressWarnings annotations.
+    (Akira Ajisaka via ozawa)
+
   OPTIMIZATIONS
 
     HADOOP-11323. WritableComparator#compare keeps reference to byte array.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/afc81888/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/find/Find.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/find/Find.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/find/Find.java
index 05cd818..70a8c79 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/find/Find.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/find/Find.java
@@ -25,6 +25,7 @@ import java.util.Deque;
 import java.util.HashSet;
 import java.util.Iterator;
 import java.util.LinkedList;
+import java.util.Set;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
@@ -63,19 +64,25 @@ public class Find extends FsCommand {
   private static final String OPTION_FOLLOW_ARG_LINK = "H";
 
   /** List of expressions recognized by this command. */
-  @SuppressWarnings("rawtypes")
-  private static final Class[] EXPRESSIONS;
+  private static final Set<Class<? extends Expression>> EXPRESSIONS =
+      new HashSet<>();
+
+  private static void addExpression(Class<?> clazz) {
+    EXPRESSIONS.add(clazz.asSubclass(Expression.class));
+  }
 
   static {
     // Initialize the static variables.
-    EXPRESSIONS = new Class[] {
-        // Operator Expressions
-        And.class,
-        // Action Expressions
-        Print.class,
-        // Navigation Expressions
-        // Matcher Expressions
-        Name.class };
+    // Operator Expressions
+    addExpression(And.class);
+
+    // Action Expressions
+    addExpression(Print.class);
+
+    // Navigation Expressions
+    // Matcher Expressions
+    addExpression(Name.class);
+
     DESCRIPTION = buildDescription(ExpressionFactory.getExpressionFactory());
 
     // Register the expressions with the expression factory.
@@ -92,7 +99,6 @@ public class Find extends FsCommand {
   private HashSet<Path> stopPaths = new HashSet<Path>();
 
   /** Register the expressions with the expression factory. */
-  @SuppressWarnings("unchecked")
   private static void registerExpressions(ExpressionFactory factory) {
     for (Class<? extends Expression> exprClass : EXPRESSIONS) {
       factory.registerExpression(exprClass);
@@ -100,7 +106,6 @@ public class Find extends FsCommand {
   }
 
   /** Build the description used by the help command. */
-  @SuppressWarnings("unchecked")
   private static String buildDescription(ExpressionFactory factory) {
     ArrayList<Expression> operators = new ArrayList<Expression>();
     ArrayList<Expression> primaries = new ArrayList<Expression>();


[19/50] [abbrv] hadoop git commit: HDFS-7819. Log WARN message for the blocks which are not in Block ID based layout (Rakesh R via Colin P. McCabe)

Posted by zh...@apache.org.
HDFS-7819. Log WARN message for the blocks which are not in Block ID based layout (Rakesh R 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/da85e17c
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/da85e17c
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/da85e17c

Branch: refs/heads/HDFS-7285
Commit: da85e17c772dfd2348b8f0e93583f7e7ac4d4e16
Parents: db0b6e6
Author: Colin Patrick Mccabe <cm...@cloudera.com>
Authored: Thu Feb 26 11:58:29 2015 -0800
Committer: Zhe Zhang <zh...@cloudera.com>
Committed: Mon Mar 2 09:13:52 2015 -0800

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |  3 +++
 .../hdfs/server/datanode/DirectoryScanner.java  | 26 +++++++++++++++++---
 2 files changed, 25 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/da85e17c/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 e09714f..54b4057 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -679,6 +679,9 @@ Release 2.7.0 - UNRELEASED
     HDFS-7832. Show 'Last Modified' in Namenode's 'Browse Filesystem'
     (vinayakumarb)
 
+    HDFS-7819. Log WARN message for the blocks which are not in Block ID based
+    layout (Rakesh R via Colin P. McCabe)
+
   OPTIMIZATIONS
 
     HDFS-7454. Reduce memory footprint for AclEntries in NameNode.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/da85e17c/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DirectoryScanner.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DirectoryScanner.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DirectoryScanner.java
index 71f976b..09c2914 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DirectoryScanner.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DirectoryScanner.java
@@ -597,14 +597,15 @@ public class DirectoryScanner implements Runnable {
       for (String bpid : bpList) {
         LinkedList<ScanInfo> report = new LinkedList<ScanInfo>();
         File bpFinalizedDir = volume.getFinalizedDir(bpid);
-        result.put(bpid, compileReport(volume, bpFinalizedDir, report));
+        result.put(bpid,
+            compileReport(volume, bpFinalizedDir, bpFinalizedDir, report));
       }
       return result;
     }
 
     /** Compile list {@link ScanInfo} for the blocks in the directory <dir> */
-    private LinkedList<ScanInfo> compileReport(FsVolumeSpi vol, File dir,
-        LinkedList<ScanInfo> report) {
+    private LinkedList<ScanInfo> compileReport(FsVolumeSpi vol,
+        File bpFinalizedDir, File dir, LinkedList<ScanInfo> report) {
       File[] files;
       try {
         files = FileUtil.listFiles(dir);
@@ -622,12 +623,14 @@ public class DirectoryScanner implements Runnable {
        */
       for (int i = 0; i < files.length; i++) {
         if (files[i].isDirectory()) {
-          compileReport(vol, files[i], report);
+          compileReport(vol, bpFinalizedDir, files[i], report);
           continue;
         }
         if (!Block.isBlockFilename(files[i])) {
           if (isBlockMetaFile("blk_", files[i].getName())) {
             long blockId = Block.getBlockId(files[i].getName());
+            verifyFileLocation(files[i].getParentFile(), bpFinalizedDir,
+                blockId);
             report.add(new ScanInfo(blockId, null, files[i], vol));
           }
           continue;
@@ -646,9 +649,24 @@ public class DirectoryScanner implements Runnable {
             break;
           }
         }
+        verifyFileLocation(blockFile.getParentFile(), bpFinalizedDir,
+            blockId);
         report.add(new ScanInfo(blockId, blockFile, metaFile, vol));
       }
       return report;
     }
+
+    /**
+     * Verify whether the actual directory location of block file has the
+     * expected directory path computed using its block ID.
+     */
+    private void verifyFileLocation(File actualBlockDir,
+        File bpFinalizedDir, long blockId) {
+      File blockDir = DatanodeUtil.idToBlockDir(bpFinalizedDir, blockId);
+      if (actualBlockDir.compareTo(blockDir) != 0) {
+        LOG.warn("Block: " + blockId
+            + " has to be upgraded to block ID-based layout");
+      }
+    }
   }
 }


[07/50] [abbrv] hadoop git commit: YARN-3239. WebAppProxy does not support a final tracking url which has query fragments and params. Contributed by Jian He

Posted by zh...@apache.org.
YARN-3239. WebAppProxy does not support a final tracking url which has query fragments and params. Contributed by Jian He


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

Branch: refs/heads/HDFS-7285
Commit: 4b09124d9d2b1b8bc7bd68c11490a5729d1e7bec
Parents: afc8188
Author: Jason Lowe <jl...@apache.org>
Authored: Wed Feb 25 16:14:34 2015 +0000
Committer: Zhe Zhang <zh...@cloudera.com>
Committed: Mon Mar 2 09:13:50 2015 -0800

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                 |  3 +++
 .../server/webproxy/WebAppProxyServlet.java     | 21 +++++++++++++-----
 .../server/webproxy/TestWebAppProxyServlet.java | 23 +++++++++++++++++++-
 3 files changed, 40 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/4b09124d/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index 2d11ed7..e5148eb 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -655,6 +655,9 @@ Release 2.7.0 - UNRELEASED
     YARN-3247. TestQueueMappings should use CapacityScheduler explicitly.
     (Zhihai Xu via ozawa)
 
+    YARN-3239. WebAppProxy does not support a final tracking url which has
+    query fragments and params (Jian He via jlowe)
+
 Release 2.6.0 - 2014-11-18
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4b09124d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-web-proxy/src/main/java/org/apache/hadoop/yarn/server/webproxy/WebAppProxyServlet.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-web-proxy/src/main/java/org/apache/hadoop/yarn/server/webproxy/WebAppProxyServlet.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-web-proxy/src/main/java/org/apache/hadoop/yarn/server/webproxy/WebAppProxyServlet.java
index 6c0391f..47f7769 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-web-proxy/src/main/java/org/apache/hadoop/yarn/server/webproxy/WebAppProxyServlet.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-web-proxy/src/main/java/org/apache/hadoop/yarn/server/webproxy/WebAppProxyServlet.java
@@ -20,9 +20,9 @@ package org.apache.hadoop.yarn.server.webproxy;
 
 import java.io.IOException;
 import java.io.InputStream;
+import java.io.ObjectInputStream;
 import java.io.OutputStream;
 import java.io.PrintWriter;
-import java.io.ObjectInputStream;
 import java.net.InetAddress;
 import java.net.URI;
 import java.net.URISyntaxException;
@@ -38,6 +38,7 @@ import javax.servlet.http.Cookie;
 import javax.servlet.http.HttpServlet;
 import javax.servlet.http.HttpServletRequest;
 import javax.servlet.http.HttpServletResponse;
+import javax.ws.rs.core.UriBuilder;
 
 import org.apache.commons.httpclient.Header;
 import org.apache.commons.httpclient.HostConfiguration;
@@ -58,6 +59,8 @@ import org.apache.hadoop.yarn.util.TrackingUriPlugin;
 import org.apache.hadoop.yarn.webapp.MimeType;
 import org.apache.hadoop.yarn.webapp.hamlet.Hamlet;
 import org.apache.hadoop.yarn.webapp.util.WebAppUtils;
+import org.apache.http.NameValuePair;
+import org.apache.http.client.utils.URLEncodedUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -325,11 +328,17 @@ public class WebAppProxyServlet extends HttpServlet {
             req.getQueryString(), true), runningUser, id);
         return;
       }
-      URI toFetch = new URI(trackingUri.getScheme(), 
-          trackingUri.getAuthority(),
-          StringHelper.ujoin(trackingUri.getPath(), rest), req.getQueryString(),
-          null);
-      
+
+      // Append the user-provided path and query parameter to the original
+      // tracking url.
+      List<NameValuePair> queryPairs =
+          URLEncodedUtils.parse(req.getQueryString(), null);
+      UriBuilder builder = UriBuilder.fromUri(trackingUri);
+      for (NameValuePair pair : queryPairs) {
+        builder.queryParam(pair.getName(), pair.getValue());
+      }
+      URI toFetch = builder.path(rest).build();
+
       LOG.info("{} is accessing unchecked {}"
           + " which is the app master GUI of {} owned by {}",
           remoteUser, toFetch, appId, runningUser);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4b09124d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-web-proxy/src/test/java/org/apache/hadoop/yarn/server/webproxy/TestWebAppProxyServlet.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-web-proxy/src/test/java/org/apache/hadoop/yarn/server/webproxy/TestWebAppProxyServlet.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-web-proxy/src/test/java/org/apache/hadoop/yarn/server/webproxy/TestWebAppProxyServlet.java
index e55c0adb..aa6d918 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-web-proxy/src/test/java/org/apache/hadoop/yarn/server/webproxy/TestWebAppProxyServlet.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-web-proxy/src/test/java/org/apache/hadoop/yarn/server/webproxy/TestWebAppProxyServlet.java
@@ -82,7 +82,7 @@ public class TestWebAppProxyServlet {
     Context context = new Context();
     context.setContextPath("/foo");
     server.setHandler(context);
-    context.addServlet(new ServletHolder(TestServlet.class), "/bar/");
+    context.addServlet(new ServletHolder(TestServlet.class), "/bar");
     server.getConnectors()[0].setHost("localhost");
     server.start();
     originalPort = server.getConnectors()[0].getLocalPort();
@@ -181,6 +181,19 @@ public class TestWebAppProxyServlet {
       proxyConn.setRequestProperty("Cookie", "checked_application_0_0000=true");
       proxyConn.connect();
       assertEquals(HttpURLConnection.HTTP_OK, proxyConn.getResponseCode());
+
+      // test user-provided path and query parameter can be appended to the
+      // original tracking url
+      appReportFetcher.answer = 5;
+      URL clientUrl = new URL("http://localhost:" + proxyPort
+        + "/proxy/application_00_0/test/tez?x=y&h=p");
+      proxyConn = (HttpURLConnection) clientUrl.openConnection();
+      proxyConn.connect();
+      LOG.info("" + proxyConn.getURL());
+      LOG.info("ProxyConn.getHeaderField(): " +  proxyConn.getHeaderField(ProxyUtils.LOCATION));
+      assertEquals("http://localhost:" + originalPort
+          + "/foo/bar/test/tez?a=b&x=y&h=p#main", proxyConn.getURL().toString());
+
     } finally {
       proxy.close();
     }
@@ -344,6 +357,14 @@ public class TestWebAppProxyServlet {
         return result;
       } else if (answer == 4) {
         throw new ApplicationNotFoundException("Application is not found");
+      } else if (answer == 5) {
+        // test user-provided path and query parameter can be appended to the
+        // original tracking url
+        ApplicationReport result = getDefaultApplicationReport(appId);
+        result.setOriginalTrackingUrl("localhost:" + originalPort
+            + "/foo/bar?a=b#main");
+        result.setYarnApplicationState(YarnApplicationState.FINISHED);
+        return result;
       }
       return null;
     }


[15/50] [abbrv] hadoop git commit: MAPREDUCE-6223. TestJobConf#testNegativeValueForTaskVmem failures. (Varun Saxena via kasha)

Posted by zh...@apache.org.
MAPREDUCE-6223. TestJobConf#testNegativeValueForTaskVmem failures. (Varun Saxena via kasha)


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

Branch: refs/heads/HDFS-7285
Commit: 2827ce2b5361e22e5fae729c6cb5f7f89f864002
Parents: 1a90367
Author: Karthik Kambatla <ka...@apache.org>
Authored: Thu Feb 26 14:24:19 2015 -0800
Committer: Zhe Zhang <zh...@cloudera.com>
Committed: Mon Mar 2 09:13:52 2015 -0800

----------------------------------------------------------------------
 hadoop-mapreduce-project/CHANGES.txt            |  3 +++
 .../java/org/apache/hadoop/mapred/JobConf.java  |  6 ++---
 .../org/apache/hadoop/conf/TestJobConf.java     | 23 ++++++++++----------
 3 files changed, 17 insertions(+), 15 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/2827ce2b/hadoop-mapreduce-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/CHANGES.txt b/hadoop-mapreduce-project/CHANGES.txt
index d2c4ab8..6cb1754 100644
--- a/hadoop-mapreduce-project/CHANGES.txt
+++ b/hadoop-mapreduce-project/CHANGES.txt
@@ -390,6 +390,9 @@ Release 2.7.0 - UNRELEASED
     MAPREDUCE-2815. JavaDoc does not generate correctly for
     MultithreadedMapRunner. (Chris Palmer via aajisaka)
 
+    MAPREDUCE-6223. TestJobConf#testNegativeValueForTaskVmem failures. 
+    (Varun Saxena via kasha)
+
 Release 2.6.1 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2827ce2b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/JobConf.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/JobConf.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/JobConf.java
index 98a643f..315c829 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/JobConf.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/JobConf.java
@@ -1826,8 +1826,7 @@ public class JobConf extends Configuration {
   public long getMemoryForMapTask() {
     long value = getDeprecatedMemoryValue();
     if (value < 0) {
-      return getLong(JobConf.MAPRED_JOB_MAP_MEMORY_MB_PROPERTY,
-          JobContext.DEFAULT_MAP_MEMORY_MB);
+      return getMemoryRequired(TaskType.MAP);
     }
     return value;
   }
@@ -1853,8 +1852,7 @@ public class JobConf extends Configuration {
   public long getMemoryForReduceTask() {
     long value = getDeprecatedMemoryValue();
     if (value < 0) {
-      return getLong(JobConf.MAPRED_JOB_REDUCE_MEMORY_MB_PROPERTY,
-          JobContext.DEFAULT_REDUCE_MEMORY_MB);
+      return getMemoryRequired(TaskType.REDUCE);
     }
     return value;
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2827ce2b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/conf/TestJobConf.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/conf/TestJobConf.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/conf/TestJobConf.java
index e380d92..f67ba1f 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/conf/TestJobConf.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/conf/TestJobConf.java
@@ -19,9 +19,7 @@ package org.apache.hadoop.conf;
 
 import org.junit.Assert;
 import org.junit.Test;
-
 import org.apache.hadoop.mapred.JobConf;
-import org.apache.hadoop.mapred.JobContext;
 import org.apache.hadoop.mapreduce.MRJobConfig;
 
 public class TestJobConf {
@@ -87,8 +85,10 @@ public class TestJobConf {
     configuration.set("mapred.task.maxvmem" , String.valueOf(-1));
     configuration.set(MRJobConfig.MAP_MEMORY_MB,"-1");
     configuration.set(MRJobConfig.REDUCE_MEMORY_MB,"-1");
-    Assert.assertEquals(configuration.getMemoryForMapTask(),-1);
-    Assert.assertEquals(configuration.getMemoryForReduceTask(),-1);    
+    Assert.assertEquals(configuration.getMemoryForMapTask(),
+        MRJobConfig.DEFAULT_MAP_MEMORY_MB);
+    Assert.assertEquals(configuration.getMemoryForReduceTask(),
+        MRJobConfig.DEFAULT_REDUCE_MEMORY_MB);
 
     configuration = new JobConf();
     configuration.set("mapred.task.maxvmem" , String.valueOf(2*1024 * 1024));
@@ -96,7 +96,6 @@ public class TestJobConf {
     configuration.set(MRJobConfig.REDUCE_MEMORY_MB, "3");
     Assert.assertEquals(configuration.getMemoryForMapTask(),2);
     Assert.assertEquals(configuration.getMemoryForReduceTask(),2);
-    
   }
 
   /**
@@ -106,7 +105,7 @@ public class TestJobConf {
   @Test
   public void testNegativeValueForTaskVmem() {
     JobConf configuration = new JobConf();
-    
+
     configuration.set(JobConf.MAPRED_TASK_MAXVMEM_PROPERTY, "-3");
     Assert.assertEquals(MRJobConfig.DEFAULT_MAP_MEMORY_MB,
         configuration.getMemoryForMapTask());
@@ -129,10 +128,12 @@ public class TestJobConf {
         
     configuration.set(MRJobConfig.MAP_MEMORY_MB, "-5");
     configuration.set(MRJobConfig.REDUCE_MEMORY_MB, "-6");
-    Assert.assertEquals(-5, configuration.getMemoryForMapTask());
-    Assert.assertEquals(-6, configuration.getMemoryForReduceTask());
+    Assert.assertEquals(MRJobConfig.DEFAULT_MAP_MEMORY_MB,
+        configuration.getMemoryForMapTask());
+    Assert.assertEquals(MRJobConfig.DEFAULT_REDUCE_MEMORY_MB,
+        configuration.getMemoryForReduceTask());
   }
-  
+
   /**
    *   Test deprecated accessor and mutator method for mapred.task.maxvmem
    */
@@ -144,13 +145,13 @@ public class TestJobConf {
     configuration.set(MRJobConfig.MAP_MEMORY_MB, String.valueOf(300));
     configuration.set(MRJobConfig.REDUCE_MEMORY_MB, String.valueOf(-1));
     Assert.assertEquals(
-      configuration.getMaxVirtualMemoryForTask(), 300 * 1024 * 1024);
+      configuration.getMaxVirtualMemoryForTask(), 1024 * 1024 * 1024);
 
     configuration = new JobConf();
     configuration.set(MRJobConfig.MAP_MEMORY_MB, String.valueOf(-1));
     configuration.set(MRJobConfig.REDUCE_MEMORY_MB, String.valueOf(200));
     Assert.assertEquals(
-      configuration.getMaxVirtualMemoryForTask(), 200 * 1024 * 1024);
+      configuration.getMaxVirtualMemoryForTask(), 1024 * 1024 * 1024);
 
     configuration = new JobConf();
     configuration.set(MRJobConfig.MAP_MEMORY_MB, String.valueOf(-1));


[45/50] [abbrv] hadoop git commit: HADOOP-11615. Update ServiceLevelAuth.md for YARN. Contributed by Brahma Reddy Battula.

Posted by zh...@apache.org.
HADOOP-11615. Update ServiceLevelAuth.md for YARN. Contributed by Brahma Reddy Battula.


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

Branch: refs/heads/HDFS-7285
Commit: 72f2c1269bd0c89d16aef199c6eb2a45ad6a2c52
Parents: 0cc5192
Author: Akira Ajisaka <aa...@apache.org>
Authored: Sun Mar 1 22:16:06 2015 -0800
Committer: Zhe Zhang <zh...@cloudera.com>
Committed: Mon Mar 2 09:13:55 2015 -0800

----------------------------------------------------------------------
 hadoop-common-project/hadoop-common/CHANGES.txt    |  3 +++
 .../src/site/markdown/ServiceLevelAuth.md          | 17 ++++++++---------
 2 files changed, 11 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/72f2c126/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 b1a7a7d..4c0c375 100644
--- a/hadoop-common-project/hadoop-common/CHANGES.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES.txt
@@ -1025,6 +1025,9 @@ Release 2.7.0 - UNRELEASED
     HADOOP-11634. Description of webhdfs' principal/keytab should switch places
     each other. (Brahma Reddy Battula via ozawa)
 
+    HADOOP-11615. Update ServiceLevelAuth.md for YARN.
+    (Brahma Reddy Battula via aajisaka)
+
 Release 2.6.1 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/72f2c126/hadoop-common-project/hadoop-common/src/site/markdown/ServiceLevelAuth.md
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/site/markdown/ServiceLevelAuth.md b/hadoop-common-project/hadoop-common/src/site/markdown/ServiceLevelAuth.md
index ae41b47..e0017d4 100644
--- a/hadoop-common-project/hadoop-common/src/site/markdown/ServiceLevelAuth.md
+++ b/hadoop-common-project/hadoop-common/src/site/markdown/ServiceLevelAuth.md
@@ -68,10 +68,9 @@ This section lists the various Hadoop services and their configuration knobs:
 | security.datanode.protocol.acl | ACL for DatanodeProtocol, which is used by datanodes to communicate with the namenode. |
 | security.inter.datanode.protocol.acl | ACL for InterDatanodeProtocol, the inter-datanode protocol for updating generation timestamp. |
 | security.namenode.protocol.acl | ACL for NamenodeProtocol, the protocol used by the secondary namenode to communicate with the namenode. |
-| security.inter.tracker.protocol.acl | ACL for InterTrackerProtocol, used by the tasktrackers to communicate with the jobtracker. |
-| security.job.submission.protocol.acl | ACL for JobSubmissionProtocol, used by job clients to communciate with the jobtracker for job submission, querying job status etc. |
-| security.task.umbilical.protocol.acl | ACL for TaskUmbilicalProtocol, used by the map and reduce tasks to communicate with the parent tasktracker. |
-| security.refresh.policy.protocol.acl | ACL for RefreshAuthorizationPolicyProtocol, used by the dfsadmin and mradmin commands to refresh the security policy in-effect. |
+| security.job.client.protocol.acl | ACL for JobSubmissionProtocol, used by job clients to communciate with the resourcemanager for job submission, querying job status etc. |
+| security.job.task.protocol.acl | ACL for TaskUmbilicalProtocol, used by the map and reduce tasks to communicate with the parent nodemanager. |
+| security.refresh.policy.protocol.acl | ACL for RefreshAuthorizationPolicyProtocol, used by the dfsadmin and rmadmin commands to refresh the security policy in-effect. |
 | security.ha.service.protocol.acl | ACL for HAService protocol used by HAAdmin to manage the active and stand-by states of namenode. |
 
 ### Access Control Lists
@@ -98,15 +97,15 @@ If access control list is not defined for a service, the value of `security.serv
 
 ### Refreshing Service Level Authorization Configuration
 
-The service-level authorization configuration for the NameNode and JobTracker can be changed without restarting either of the Hadoop master daemons. The cluster administrator can change `$HADOOP_CONF_DIR/hadoop-policy.xml` on the master nodes and instruct the NameNode and JobTracker to reload their respective configurations via the `-refreshServiceAcl` switch to `dfsadmin` and `mradmin` commands respectively.
+The service-level authorization configuration for the NameNode and ResourceManager can be changed without restarting either of the Hadoop master daemons. The cluster administrator can change `$HADOOP_CONF_DIR/hadoop-policy.xml` on the master nodes and instruct the NameNode and ResourceManager to reload their respective configurations via the `-refreshServiceAcl` switch to `dfsadmin` and `rmadmin` commands respectively.
 
 Refresh the service-level authorization configuration for the NameNode:
 
-       $ bin/hadoop dfsadmin -refreshServiceAcl
+       $ bin/hdfs dfsadmin -refreshServiceAcl
 
-Refresh the service-level authorization configuration for the JobTracker:
+Refresh the service-level authorization configuration for the ResourceManager:
 
-       $ bin/hadoop mradmin -refreshServiceAcl
+       $ bin/yarn rmadmin -refreshServiceAcl
 
 Of course, one can use the `security.refresh.policy.protocol.acl` property in `$HADOOP_CONF_DIR/hadoop-policy.xml` to restrict access to the ability to refresh the service-level authorization configuration to certain users/groups.
 
@@ -125,7 +124,7 @@ Of course, one can use the `security.refresh.policy.protocol.acl` property in `$
 Allow only users `alice`, `bob` and users in the `mapreduce` group to submit jobs to the MapReduce cluster:
 
     <property>
-         <name>security.job.submission.protocol.acl</name>
+         <name>security.job.client.protocol.acl</name>
          <value>alice,bob mapreduce</value>
     </property>
 


[13/50] [abbrv] hadoop git commit: HDFS-7843. A truncated file is corrupted after rollback from a rolling upgrade.

Posted by zh...@apache.org.
HDFS-7843. A truncated file is corrupted after rollback from a rolling upgrade.


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

Branch: refs/heads/HDFS-7285
Commit: 746bc377b7252d344b5d83f3ae862b717085db4d
Parents: 7911e1d
Author: Tsz-Wo Nicholas Sze <sz...@hortonworks.com>
Authored: Thu Feb 26 10:14:40 2015 +0800
Committer: Zhe Zhang <zh...@cloudera.com>
Committed: Mon Mar 2 09:13:51 2015 -0800

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |  3 ++
 .../hdfs/server/namenode/FSNamesystem.java      |  3 ++
 .../apache/hadoop/hdfs/TestRollingUpgrade.java  | 48 ++++++++++++++------
 3 files changed, 40 insertions(+), 14 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/746bc377/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 e0f9267..f8b0c37 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -1026,6 +1026,9 @@ Release 2.7.0 - UNRELEASED
     HDFS-7763. fix zkfc hung issue due to not catching exception in a corner
     case. (Liang Xie via wang)
 
+    HDFS-7843. A truncated file is corrupted after rollback from a rolling
+    upgrade.  (szetszwo)
+
     BREAKDOWN OF HDFS-7584 SUBTASKS AND RELATED JIRAS
 
       HDFS-7720. Quota by Storage Type API, tools and ClientNameNode

http://git-wip-us.apache.org/repos/asf/hadoop/blob/746bc377/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 deaf90c..63ffd81 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
@@ -2089,6 +2089,9 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     if(!isUpgradeFinalized()) {
       return true;
     }
+    if (isRollingUpgrade()) {
+      return true;
+    }
     return file.isBlockInLatestSnapshot(blk);
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/746bc377/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestRollingUpgrade.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestRollingUpgrade.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestRollingUpgrade.java
index 8e7b4b1..9746049 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestRollingUpgrade.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestRollingUpgrade.java
@@ -23,9 +23,11 @@ import java.io.IOException;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.MiniDFSCluster.DataNodeProperties;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.RollingUpgradeAction;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
 import org.apache.hadoop.hdfs.protocol.RollingUpgradeInfo;
@@ -36,6 +38,7 @@ import org.apache.hadoop.hdfs.server.datanode.DataNode;
 import org.apache.hadoop.hdfs.server.namenode.FSImage;
 import org.apache.hadoop.hdfs.server.namenode.NNStorage;
 import org.apache.hadoop.hdfs.server.namenode.SecondaryNameNode;
+import org.apache.hadoop.hdfs.server.namenode.TestFileTruncate;
 import org.apache.hadoop.hdfs.tools.DFSAdmin;
 import org.apache.hadoop.io.IOUtils;
 import org.junit.Assert;
@@ -260,42 +263,50 @@ public class TestRollingUpgrade {
     final Configuration conf = new HdfsConfiguration();
     MiniDFSCluster cluster = null;
     try {
-      cluster = new MiniDFSCluster.Builder(conf).numDataNodes(0).build();
+      cluster = new MiniDFSCluster.Builder(conf).numDataNodes(1).build();
       cluster.waitActive();
 
       final Path foo = new Path("/foo");
       final Path bar = new Path("/bar");
       cluster.getFileSystem().mkdirs(foo);
 
-      startRollingUpgrade(foo, bar, cluster);
+      final Path file = new Path(foo, "file");
+      final byte[] data = new byte[1024];
+      DFSUtil.getRandom().nextBytes(data);
+      final FSDataOutputStream out = cluster.getFileSystem().create(file);
+      out.write(data, 0, data.length);
+      out.close();
+
+      startRollingUpgrade(foo, bar, file, data, cluster);
       cluster.getFileSystem().rollEdits();
       cluster.getFileSystem().rollEdits();
-      rollbackRollingUpgrade(foo, bar, cluster);
+      rollbackRollingUpgrade(foo, bar, file, data, cluster);
 
-      startRollingUpgrade(foo, bar, cluster);
+      startRollingUpgrade(foo, bar, file, data, cluster);
       cluster.getFileSystem().rollEdits();
       cluster.getFileSystem().rollEdits();
-      rollbackRollingUpgrade(foo, bar, cluster);
+      rollbackRollingUpgrade(foo, bar, file, data, cluster);
 
-      startRollingUpgrade(foo, bar, cluster);
+      startRollingUpgrade(foo, bar, file, data, cluster);
       cluster.restartNameNode();
-      rollbackRollingUpgrade(foo, bar, cluster);
+      rollbackRollingUpgrade(foo, bar, file, data, cluster);
 
-      startRollingUpgrade(foo, bar, cluster);
+      startRollingUpgrade(foo, bar, file, data, cluster);
       cluster.restartNameNode();
-      rollbackRollingUpgrade(foo, bar, cluster);
+      rollbackRollingUpgrade(foo, bar, file, data, cluster);
 
-      startRollingUpgrade(foo, bar, cluster);
-      rollbackRollingUpgrade(foo, bar, cluster);
+      startRollingUpgrade(foo, bar, file, data, cluster);
+      rollbackRollingUpgrade(foo, bar, file, data, cluster);
 
-      startRollingUpgrade(foo, bar, cluster);
-      rollbackRollingUpgrade(foo, bar, cluster);
+      startRollingUpgrade(foo, bar, file, data, cluster);
+      rollbackRollingUpgrade(foo, bar, file, data, cluster);
     } finally {
       if(cluster != null) cluster.shutdown();
     }
   }
   
   private static void startRollingUpgrade(Path foo, Path bar,
+      Path file, byte[] data,
       MiniDFSCluster cluster) throws IOException {
     final DistributedFileSystem dfs = cluster.getFileSystem();
 
@@ -305,18 +316,27 @@ public class TestRollingUpgrade {
     dfs.setSafeMode(SafeModeAction.SAFEMODE_LEAVE);
 
     dfs.mkdirs(bar);
-    
     Assert.assertTrue(dfs.exists(foo));
     Assert.assertTrue(dfs.exists(bar));
+
+    //truncate a file
+    final int newLength = DFSUtil.getRandom().nextInt(data.length - 1) + 1;
+    dfs.truncate(file, newLength);
+    TestFileTruncate.checkBlockRecovery(file, dfs);
+    AppendTestUtil.checkFullFile(dfs, file, newLength, data);
   }
   
   private static void rollbackRollingUpgrade(Path foo, Path bar,
+      Path file, byte[] data,
       MiniDFSCluster cluster) throws IOException {
+    final DataNodeProperties dnprop = cluster.stopDataNode(0);
     cluster.restartNameNode("-rollingUpgrade", "rollback");
+    cluster.restartDataNode(dnprop, true);
 
     final DistributedFileSystem dfs = cluster.getFileSystem();
     Assert.assertTrue(dfs.exists(foo));
     Assert.assertFalse(dfs.exists(bar));
+    AppendTestUtil.checkFullFile(dfs, file, data.length, data);
   }
 
   @Test


[43/50] [abbrv] hadoop git commit: HADOOP-11634. Description of webhdfs' principal/keytab should switch places each other. Contributed by Brahma Reddy Battula.

Posted by zh...@apache.org.
HADOOP-11634. Description of webhdfs' principal/keytab should switch places each other. Contributed by Brahma Reddy Battula.


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

Branch: refs/heads/HDFS-7285
Commit: 731a463ed13339e71becba7fa6a1110a6ab33704
Parents: b1ddc71
Author: Tsuyoshi Ozawa <oz...@apache.org>
Authored: Mon Mar 2 04:18:07 2015 +0900
Committer: Zhe Zhang <zh...@cloudera.com>
Committed: Mon Mar 2 09:13:55 2015 -0800

----------------------------------------------------------------------
 hadoop-common-project/hadoop-common/CHANGES.txt                  | 3 +++
 .../hadoop-common/src/site/markdown/SecureMode.md                | 4 ++--
 2 files changed, 5 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/731a463e/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 3c4dc99..f1d48bc 100644
--- a/hadoop-common-project/hadoop-common/CHANGES.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES.txt
@@ -1019,6 +1019,9 @@ Release 2.7.0 - UNRELEASED
     HADOOP-9922. hadoop windows native build will fail in 32 bit machine.
     (Kiran Kumar M R via cnauroth)
 
+    HADOOP-11634. Description of webhdfs' principal/keytab should switch places
+    each other. (Brahma Reddy Battula via ozawa)
+
 Release 2.6.1 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/731a463e/hadoop-common-project/hadoop-common/src/site/markdown/SecureMode.md
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/site/markdown/SecureMode.md b/hadoop-common-project/hadoop-common/src/site/markdown/SecureMode.md
index 0004d25..cb27e29 100644
--- a/hadoop-common-project/hadoop-common/src/site/markdown/SecureMode.md
+++ b/hadoop-common-project/hadoop-common/src/site/markdown/SecureMode.md
@@ -289,8 +289,8 @@ The following properties should be in the `core-site.xml` of all the nodes in th
 
 | Parameter | Value | Notes |
 |:---- |:---- |:---- |
-| `dfs.web.authentication.kerberos.principal` | http/\_HOST@REALM.TLD | Kerberos keytab file for the WebHDFS. |
-| `dfs.web.authentication.kerberos.keytab` | */etc/security/keytab/http.service.keytab* | Kerberos principal name for WebHDFS. |
+| `dfs.web.authentication.kerberos.principal` | http/\_HOST@REALM.TLD | Kerberos principal name for the WebHDFS. |
+| `dfs.web.authentication.kerberos.keytab` | */etc/security/keytab/http.service.keytab* | Kerberos keytab file for WebHDFS. |
 
 ### ResourceManager
 


[44/50] [abbrv] hadoop git commit: HADOOP-11657. Align the output of `hadoop fs -du` to be more Unix-like. (aajisaka)

Posted by zh...@apache.org.
HADOOP-11657. Align the output of `hadoop fs -du` to be more Unix-like. (aajisaka)


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

Branch: refs/heads/HDFS-7285
Commit: 0cc5192520bb4ed9551888fb24206b6a29e2db6a
Parents: 731a463
Author: Akira Ajisaka <aa...@apache.org>
Authored: Sun Mar 1 21:09:15 2015 -0800
Committer: Zhe Zhang <zh...@cloudera.com>
Committed: Mon Mar 2 09:13:55 2015 -0800

----------------------------------------------------------------------
 hadoop-common-project/hadoop-common/CHANGES.txt |  3 ++
 .../org/apache/hadoop/fs/shell/FsUsage.java     | 12 ++++++--
 .../org/apache/hadoop/hdfs/TestDFSShell.java    | 29 ++++++++++++++++++++
 3 files changed, 42 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/0cc51925/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 f1d48bc..b1a7a7d 100644
--- a/hadoop-common-project/hadoop-common/CHANGES.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES.txt
@@ -13,6 +13,9 @@ Trunk (Unreleased)
 
     HADOOP-10950. rework heap management vars (John Smith via aw)
 
+    HADOOP-11657. Align the output of `hadoop fs -du` to be more Unix-like.
+    (aajisaka)
+
   NEW FEATURES
 
     HADOOP-6590. Add a username check for hadoop sub-commands (John Smith via aw)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0cc51925/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/FsUsage.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/FsUsage.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/FsUsage.java
index 5c1dbf0..765b181 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/FsUsage.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/FsUsage.java
@@ -132,15 +132,23 @@ class FsUsage extends FsCommand {
     }
 
     @Override
-    protected void processPathArgument(PathData item) throws IOException {
+    protected void processArguments(LinkedList<PathData> args)
+        throws IOException {
       usagesTable = new TableBuilder(3);
+      super.processArguments(args);
+      if (!usagesTable.isEmpty()) {
+        usagesTable.printToStream(out);
+      }
+    }
+
+    @Override
+    protected void processPathArgument(PathData item) throws IOException {
       // go one level deep on dirs from cmdline unless in summary mode
       if (!summary && item.stat.isDirectory()) {
         recursePath(item);
       } else {
         super.processPathArgument(item);
       }
-      usagesTable.printToStream(out);
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0cc51925/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSShell.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSShell.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSShell.java
index ee04076..0a88208 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSShell.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSShell.java
@@ -95,6 +95,14 @@ public class TestDFSShell {
     return f;
   }
 
+  static Path writeByte(FileSystem fs, Path f) throws IOException {
+    DataOutputStream out = fs.create(f);
+    out.writeByte(1);
+    out.close();
+    assertTrue(fs.exists(f));
+    return f;
+  }
+
   static Path mkdir(FileSystem fs, Path p) throws IOException {
     assertTrue(fs.mkdirs(p));
     assertTrue(fs.exists(p));
@@ -272,6 +280,27 @@ public class TestDFSShell {
       Long combinedDiskUsed = myFileDiskUsed + myFile2DiskUsed;
       assertThat(returnString, containsString(combinedLength.toString()));
       assertThat(returnString, containsString(combinedDiskUsed.toString()));
+
+      // Check if output is rendered properly with multiple input paths
+      Path myFile3 = new Path("/test/dir/file3");
+      writeByte(fs, myFile3);
+      assertTrue(fs.exists(myFile3));
+      args = new String[3];
+      args[0] = "-du";
+      args[1] = "/test/dir/file3";
+      args[2] = "/test/dir/file2";
+      val = -1;
+      try {
+        val = shell.run(args);
+      } catch (Exception e) {
+        System.err.println("Exception raised from DFSShell.run " +
+            e.getLocalizedMessage());
+      }
+      assertEquals("Return code should be 0.", 0, val);
+      returnString = out.toString();
+      out.reset();
+      assertTrue(returnString.contains("1   2   /test/dir/file3"));
+      assertTrue(returnString.contains("23  46  /test/dir/file2"));
     } finally {
       System.setOut(psBackup);
       cluster.shutdown();


[05/50] [abbrv] hadoop git commit: HADOOP-11480. Typo in hadoop-aws/index.md uses wrong scheme for test.fs.s3.name. Contributed by Ted Yu.

Posted by zh...@apache.org.
HADOOP-11480. Typo in hadoop-aws/index.md uses wrong scheme for test.fs.s3.name. Contributed by Ted Yu.


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

Branch: refs/heads/HDFS-7285
Commit: ef3702ef300b5d897592d9d9a5990282c3e0f36a
Parents: fe66fe4
Author: Akira Ajisaka <aa...@apache.org>
Authored: Tue Feb 24 17:11:46 2015 -0800
Committer: Zhe Zhang <zh...@cloudera.com>
Committed: Mon Mar 2 09:13:50 2015 -0800

----------------------------------------------------------------------
 hadoop-common-project/hadoop-common/CHANGES.txt                  | 3 +++
 .../hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md       | 4 ++--
 2 files changed, 5 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/ef3702ef/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 96d5724..a5a11b9 100644
--- a/hadoop-common-project/hadoop-common/CHANGES.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES.txt
@@ -996,6 +996,9 @@ Release 2.7.0 - UNRELEASED
     HADOOP-11619. FTPFileSystem should override getDefaultPort.
     (Brahma Reddy Battula via gera)
 
+    HADOOP-11480. Typo in hadoop-aws/index.md uses wrong scheme for
+    test.fs.s3.name. (Ted Yu via aajisaka)
+
 Release 2.6.1 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ef3702ef/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md
index 1e44864..8e80b92 100644
--- a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md
+++ b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md
@@ -353,7 +353,7 @@ Example:
     
       <property>
         <name>test.fs.s3.name</name>
-        <value>s3a://test-aws-s3/</value>
+        <value>s3://test-aws-s3/</value>
       </property>
   
       <property>
@@ -493,4 +493,4 @@ Example:
 This example pulls in the `auth-keys.xml` file for the credentials. 
 This provides one single place to keep the keys up to date —and means
 that the file `contract-test-options.xml` does not contain any
-secret credentials itself.
\ No newline at end of file
+secret credentials itself.


[28/50] [abbrv] hadoop git commit: YARN-3262. Surface application outstanding resource requests table in RM web UI. (Jian He via wangda)

Posted by zh...@apache.org.
YARN-3262. Surface application outstanding resource requests table in RM web UI. (Jian He via wangda)


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

Branch: refs/heads/HDFS-7285
Commit: eaccaba3121c6fcebc8b4ab94ec48ee1085a3fb5
Parents: 97c2c59
Author: Wangda Tan <wa...@apache.org>
Authored: Fri Feb 27 16:13:32 2015 -0800
Committer: Zhe Zhang <zh...@cloudera.com>
Committed: Mon Mar 2 09:13:54 2015 -0800

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                 |  3 ++
 .../records/impl/pb/ResourceRequestPBImpl.java  |  4 +-
 .../scheduler/AbstractYarnScheduler.java        |  9 ++++
 .../scheduler/AppSchedulingInfo.java            | 33 +++++++-------
 .../scheduler/SchedulerApplicationAttempt.java  |  6 ++-
 .../server/resourcemanager/webapp/AppBlock.java | 46 +++++++++++++++++++-
 .../server/resourcemanager/webapp/AppPage.java  |  4 ++
 .../resourcemanager/webapp/AppsBlock.java       |  5 ++-
 .../webapp/FairSchedulerAppsBlock.java          |  5 ++-
 .../resourcemanager/webapp/RMWebServices.java   |  6 +--
 .../resourcemanager/webapp/dao/AppInfo.java     | 17 +++++++-
 .../webapp/TestRMWebAppFairScheduler.java       | 10 ++++-
 .../webapp/TestRMWebServicesApps.java           |  3 +-
 13 files changed, 118 insertions(+), 33 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/eaccaba3/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index 38dd9fa..e7af84b 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -336,6 +336,9 @@ Release 2.7.0 - UNRELEASED
     YARN-2820. Retry in FileSystemRMStateStore when FS's operations fail 
     due to IOException. (Zhihai Xu via ozawa)
 
+    YARN-3262. Surface application outstanding resource requests table 
+    in RM web UI. (Jian He via wangda)
+
   OPTIMIZATIONS
 
     YARN-2990. FairScheduler's delay-scheduling always waits for node-local and 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/eaccaba3/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ResourceRequestPBImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ResourceRequestPBImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ResourceRequestPBImpl.java
index 0c8491f..27fb5ae 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ResourceRequestPBImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ResourceRequestPBImpl.java
@@ -140,13 +140,13 @@ public class ResourceRequestPBImpl extends  ResourceRequest {
     this.capability = capability;
   }
   @Override
-  public int getNumContainers() {
+  public synchronized int getNumContainers() {
     ResourceRequestProtoOrBuilder p = viaProto ? proto : builder;
     return (p.getNumContainers());
   }
 
   @Override
-  public void setNumContainers(int numContainers) {
+  public synchronized void setNumContainers(int numContainers) {
     maybeInitBuilder();
     builder.setNumContainers((numContainers));
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/eaccaba3/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AbstractYarnScheduler.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AbstractYarnScheduler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AbstractYarnScheduler.java
index 04b3452..968a767 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AbstractYarnScheduler.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AbstractYarnScheduler.java
@@ -658,4 +658,13 @@ public abstract class AbstractYarnScheduler
       maxAllocWriteLock.unlock();
     }
   }
+
+  public List<ResourceRequest> getPendingResourceRequestsForAttempt(
+      ApplicationAttemptId attemptId) {
+    SchedulerApplicationAttempt attempt = getApplicationAttempt(attemptId);
+    if (attempt != null) {
+      return attempt.getAppSchedulingInfo().getAllResourceRequests();
+    }
+    return null;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/eaccaba3/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AppSchedulingInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AppSchedulingInfo.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AppSchedulingInfo.java
index a9a459f..97dc231 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AppSchedulingInfo.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AppSchedulingInfo.java
@@ -20,12 +20,14 @@ package org.apache.hadoop.yarn.server.resourcemanager.scheduler;
 
 import java.util.ArrayList;
 import java.util.Collection;
+import java.util.Collections;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
 import java.util.TreeSet;
+import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.atomic.AtomicLong;
 
 import org.apache.commons.logging.Log;
@@ -64,7 +66,7 @@ public class AppSchedulingInfo {
   final Set<Priority> priorities = new TreeSet<Priority>(
       new org.apache.hadoop.yarn.server.resourcemanager.resource.Priority.Comparator());
   final Map<Priority, Map<String, ResourceRequest>> requests =
-    new HashMap<Priority, Map<String, ResourceRequest>>();
+    new ConcurrentHashMap<Priority, Map<String, ResourceRequest>>();
   private Set<String> blacklist = new HashSet<String>();
 
   //private final ApplicationStore store;
@@ -159,7 +161,7 @@ public class AppSchedulingInfo {
       Map<String, ResourceRequest> asks = this.requests.get(priority);
 
       if (asks == null) {
-        asks = new HashMap<String, ResourceRequest>();
+        asks = new ConcurrentHashMap<String, ResourceRequest>();
         this.requests.put(priority, asks);
         this.priorities.add(priority);
       }
@@ -221,7 +223,7 @@ public class AppSchedulingInfo {
     return requests.get(priority);
   }
 
-  synchronized public List<ResourceRequest> getAllResourceRequests() {
+  public List<ResourceRequest> getAllResourceRequests() {
     List<ResourceRequest> ret = new ArrayList<ResourceRequest>();
     for (Map<String, ResourceRequest> r : requests.values()) {
       ret.addAll(r.values());
@@ -300,17 +302,11 @@ public class AppSchedulingInfo {
       Priority priority, ResourceRequest nodeLocalRequest, Container container,
       List<ResourceRequest> resourceRequests) {
     // Update future requirements
-    nodeLocalRequest.setNumContainers(nodeLocalRequest.getNumContainers() - 1);
-    if (nodeLocalRequest.getNumContainers() == 0) {
-      this.requests.get(priority).remove(node.getNodeName());
-    }
+    decResourceRequest(node.getNodeName(), priority, nodeLocalRequest);
 
     ResourceRequest rackLocalRequest = requests.get(priority).get(
         node.getRackName());
-    rackLocalRequest.setNumContainers(rackLocalRequest.getNumContainers() - 1);
-    if (rackLocalRequest.getNumContainers() == 0) {
-      this.requests.get(priority).remove(node.getRackName());
-    }
+    decResourceRequest(node.getRackName(), priority, rackLocalRequest);
 
     ResourceRequest offRackRequest = requests.get(priority).get(
         ResourceRequest.ANY);
@@ -322,6 +318,14 @@ public class AppSchedulingInfo {
     resourceRequests.add(cloneResourceRequest(offRackRequest));
   }
 
+  private void decResourceRequest(String resourceName, Priority priority,
+      ResourceRequest request) {
+    request.setNumContainers(request.getNumContainers() - 1);
+    if (request.getNumContainers() == 0) {
+      requests.get(priority).remove(resourceName);
+    }
+  }
+
   /**
    * The {@link ResourceScheduler} is allocating data-local resources to the
    * application.
@@ -333,11 +337,8 @@ public class AppSchedulingInfo {
       Priority priority, ResourceRequest rackLocalRequest, Container container,
       List<ResourceRequest> resourceRequests) {
     // Update future requirements
-    rackLocalRequest.setNumContainers(rackLocalRequest.getNumContainers() - 1);
-    if (rackLocalRequest.getNumContainers() == 0) {
-      this.requests.get(priority).remove(node.getRackName());
-    }
-
+    decResourceRequest(node.getRackName(), priority, rackLocalRequest);
+    
     ResourceRequest offRackRequest = requests.get(priority).get(
         ResourceRequest.ANY);
     decrementOutstanding(offRackRequest);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/eaccaba3/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerApplicationAttempt.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerApplicationAttempt.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerApplicationAttempt.java
index d5b6ce6..532df05 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerApplicationAttempt.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerApplicationAttempt.java
@@ -153,7 +153,11 @@ public class SchedulerApplicationAttempt {
   public synchronized Collection<RMContainer> getLiveContainers() {
     return new ArrayList<RMContainer>(liveContainers.values());
   }
-  
+
+  public AppSchedulingInfo getAppSchedulingInfo() {
+    return this.appSchedulingInfo;
+  }
+
   /**
    * Is this application pending?
    * @return true if it is else false.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/eaccaba3/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/AppBlock.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/AppBlock.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/AppBlock.java
index c2b376e..62ad8df 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/AppBlock.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/AppBlock.java
@@ -35,6 +35,7 @@ import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
 import org.apache.hadoop.yarn.api.records.QueueACL;
 import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.api.records.ResourceRequest;
 import org.apache.hadoop.yarn.api.records.YarnApplicationState;
 import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
 import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager;
@@ -50,6 +51,7 @@ import org.apache.hadoop.yarn.util.resource.Resources;
 import org.apache.hadoop.yarn.webapp.hamlet.Hamlet;
 import org.apache.hadoop.yarn.webapp.hamlet.Hamlet.DIV;
 import org.apache.hadoop.yarn.webapp.hamlet.Hamlet.TABLE;
+import org.apache.hadoop.yarn.webapp.hamlet.Hamlet.TBODY;
 import org.apache.hadoop.yarn.webapp.util.WebAppUtils;
 import org.apache.hadoop.yarn.webapp.view.HtmlBlock;
 import org.apache.hadoop.yarn.webapp.view.InfoBlock;
@@ -90,7 +92,8 @@ public class AppBlock extends HtmlBlock {
       puts("Application not found: "+ aid);
       return;
     }
-    AppInfo app = new AppInfo(rmApp, true, WebAppUtils.getHttpSchemePrefix(conf));
+    AppInfo app =
+        new AppInfo(rm, rmApp, true, WebAppUtils.getHttpSchemePrefix(conf));
 
     // Check for the authorization.
     String remoteUser = request().getRemoteUser();
@@ -134,7 +137,7 @@ public class AppBlock extends HtmlBlock {
         ._("Application Type:", app.getApplicationType())
         ._("Application Tags:", app.getApplicationTags())
         ._("YarnApplicationState:", clarifyAppState(app.getState()))
-        ._("FinalStatus reported by AM:",
+        ._("FinalStatus Reported by AM:",
           clairfyAppFinalStatus(app.getFinalStatus()))
         ._("Started:", Times.format(app.getStartTime()))
         ._("Elapsed:",
@@ -200,6 +203,45 @@ public class AppBlock extends HtmlBlock {
 
     table._();
     div._();
+
+    createResourceRequestsTable(html, app);
+  }
+
+  private void createResourceRequestsTable(Block html, AppInfo app) {
+    TBODY<TABLE<Hamlet>> tbody =
+        html.table("#ResourceRequests").thead().tr()
+          .th(".priority", "Priority")
+          .th(".resourceName", "ResourceName")
+          .th(".totalResource", "Capability")
+          .th(".numContainers", "NumContainers")
+          .th(".relaxLocality", "RelaxLocality")
+          .th(".nodeLabelExpression", "NodeLabelExpression")._()._().tbody();
+
+    Resource totalResource = Resource.newInstance(0, 0);
+    if (app.getResourceRequests() != null) {
+      for (ResourceRequest request : app.getResourceRequests()) {
+        if (request.getNumContainers() == 0) {
+          continue;
+        }
+
+        tbody.tr()
+          .td(String.valueOf(request.getPriority()))
+          .td(request.getResourceName())
+          .td(String.valueOf(request.getCapability()))
+          .td(String.valueOf(request.getNumContainers()))
+          .td(String.valueOf(request.getRelaxLocality()))
+          .td(request.getNodeLabelExpression() == null ? "N/A" : request
+              .getNodeLabelExpression())._();
+        if (request.getResourceName().equals(ResourceRequest.ANY)) {
+          Resources.addTo(totalResource,
+            Resources.multiply(request.getCapability(),
+              request.getNumContainers()));
+        }
+      }
+    }
+    html.div().$class("totalResourceRequests")
+      .h3("Total Outstanding Resource Requests: " + totalResource)._();
+    tbody._()._();
   }
 
   private String clarifyAppState(YarnApplicationState state) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/eaccaba3/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/AppPage.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/AppPage.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/AppPage.java
index a55c62f..8993324 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/AppPage.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/AppPage.java
@@ -18,12 +18,16 @@
 
 package org.apache.hadoop.yarn.server.resourcemanager.webapp;
 
+import static org.apache.hadoop.yarn.webapp.view.JQueryUI.DATATABLES_ID;
+
 import org.apache.hadoop.yarn.webapp.SubView;
 
 public class AppPage extends RmView {
 
   @Override protected void preHead(Page.HTML<_> html) {
     commonPreHead(html);
+    set(DATATABLES_ID, "ResourceRequests");
+    setTableStyles(html, "ResourceRequests");
   }
 
   @Override protected Class<? extends SubView> content() {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/eaccaba3/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/AppsBlock.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/AppsBlock.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/AppsBlock.java
index 054a1a7..935be61 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/AppsBlock.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/AppsBlock.java
@@ -46,12 +46,13 @@ import com.google.inject.Inject;
 class AppsBlock extends HtmlBlock {
   final ConcurrentMap<ApplicationId, RMApp> apps;
   private final Configuration conf;
-
+  final ResourceManager rm;
   @Inject
   AppsBlock(ResourceManager rm, ViewContext ctx, Configuration conf) {
     super(ctx);
     apps = rm.getRMContext().getRMApps();
     this.conf = conf;
+    this.rm = rm;
   }
 
   @Override public void render(Block html) {
@@ -85,7 +86,7 @@ class AppsBlock extends HtmlBlock {
       if (reqAppStates != null && !reqAppStates.contains(app.createApplicationState())) {
         continue;
       }
-      AppInfo appInfo = new AppInfo(app, true, WebAppUtils.getHttpSchemePrefix(conf));
+      AppInfo appInfo = new AppInfo(rm, app, true, WebAppUtils.getHttpSchemePrefix(conf));
       String percent = String.format("%.1f", appInfo.getProgress());
       //AppID numerical value parsed by parseHadoopID in yarn.dt.plugins.js
       appsTableData.append("[\"<a href='")

http://git-wip-us.apache.org/repos/asf/hadoop/blob/eaccaba3/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/FairSchedulerAppsBlock.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/FairSchedulerAppsBlock.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/FairSchedulerAppsBlock.java
index 42ee53c..8cfd246 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/FairSchedulerAppsBlock.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/FairSchedulerAppsBlock.java
@@ -56,7 +56,7 @@ public class FairSchedulerAppsBlock extends HtmlBlock {
   final ConcurrentMap<ApplicationId, RMApp> apps;
   final FairSchedulerInfo fsinfo;
   final Configuration conf;
-  
+  final ResourceManager rm;
   @Inject
   public FairSchedulerAppsBlock(ResourceManager rm, ViewContext ctx,
       Configuration conf) {
@@ -73,6 +73,7 @@ public class FairSchedulerAppsBlock extends HtmlBlock {
       }
     }
     this.conf = conf;
+    this.rm = rm;
   }
   
   @Override public void render(Block html) {
@@ -107,7 +108,7 @@ public class FairSchedulerAppsBlock extends HtmlBlock {
       if (reqAppStates != null && !reqAppStates.contains(app.createApplicationState())) {
         continue;
       }
-      AppInfo appInfo = new AppInfo(app, true, WebAppUtils.getHttpSchemePrefix(conf));
+      AppInfo appInfo = new AppInfo(rm, app, true, WebAppUtils.getHttpSchemePrefix(conf));
       String percent = String.format("%.1f", appInfo.getProgress());
       ApplicationAttemptId attemptId = app.getCurrentAppAttempt().getAppAttemptId();
       int fairShare = fsinfo.getAppFairShare(attemptId);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/eaccaba3/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMWebServices.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMWebServices.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMWebServices.java
index 1834b6a..f8836d5 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMWebServices.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMWebServices.java
@@ -476,8 +476,8 @@ public class RMWebServices {
         }
       }
 
-      AppInfo app = new AppInfo(rmapp, hasAccess(rmapp, hsr),
-          WebAppUtils.getHttpSchemePrefix(conf));
+      AppInfo app = new AppInfo(rm, rmapp,
+          hasAccess(rmapp, hsr), WebAppUtils.getHttpSchemePrefix(conf));
       allApps.add(app);
     }
     return allApps;
@@ -617,7 +617,7 @@ public class RMWebServices {
     if (app == null) {
       throw new NotFoundException("app with id: " + appId + " not found");
     }
-    return new AppInfo(app, hasAccess(app, hsr), hsr.getScheme() + "://");
+    return new AppInfo(rm, app, hasAccess(app, hsr), hsr.getScheme() + "://");
   }
 
   @GET

http://git-wip-us.apache.org/repos/asf/hadoop/blob/eaccaba3/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/AppInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/AppInfo.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/AppInfo.java
index 66940cb..79b2248 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/AppInfo.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/AppInfo.java
@@ -17,6 +17,8 @@
  */
 package org.apache.hadoop.yarn.server.resourcemanager.webapp.dao;
 
+import java.util.List;
+
 import javax.xml.bind.annotation.XmlAccessType;
 import javax.xml.bind.annotation.XmlAccessorType;
 import javax.xml.bind.annotation.XmlRootElement;
@@ -27,11 +29,13 @@ import org.apache.hadoop.yarn.api.records.ApplicationResourceUsageReport;
 import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
 import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.api.records.ResourceRequest;
 import org.apache.hadoop.yarn.api.records.YarnApplicationState;
 import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppMetrics;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttempt;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.AbstractYarnScheduler;
 import org.apache.hadoop.yarn.util.ConverterUtils;
 import org.apache.hadoop.yarn.util.Times;
 import org.apache.hadoop.yarn.webapp.util.WebAppUtils;
@@ -88,10 +92,14 @@ public class AppInfo {
   protected int numNonAMContainerPreempted;
   protected int numAMContainerPreempted;
 
+  protected List<ResourceRequest> resourceRequests;
+
   public AppInfo() {
   } // JAXB needs this
 
-  public AppInfo(RMApp app, Boolean hasAccess, String schemePrefix) {
+  @SuppressWarnings({ "rawtypes", "unchecked" })
+  public AppInfo(ResourceManager rm, RMApp app, Boolean hasAccess,
+      String schemePrefix) {
     this.schemePrefix = schemePrefix;
     if (app != null) {
       String trackingUrl = app.getTrackingUrl();
@@ -154,6 +162,9 @@ public class AppInfo {
             allocatedVCores = usedResources.getVirtualCores();
             runningContainers = resourceReport.getNumUsedContainers();
           }
+          resourceRequests =
+              ((AbstractYarnScheduler) rm.getRMContext().getScheduler())
+                .getPendingResourceRequestsForAttempt(attempt.getAppAttemptId());
         }
       }
 
@@ -299,4 +310,8 @@ public class AppInfo {
   public long getVcoreSeconds() {
     return vcoreSeconds;
   }
+
+  public List<ResourceRequest> getResourceRequests() {
+    return this.resourceRequests;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/eaccaba3/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebAppFairScheduler.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebAppFairScheduler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebAppFairScheduler.java
index f07cb8d..b850a5e 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebAppFairScheduler.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebAppFairScheduler.java
@@ -22,6 +22,7 @@ import com.google.common.collect.Maps;
 import com.google.inject.Binder;
 import com.google.inject.Injector;
 import com.google.inject.Module;
+
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
@@ -35,8 +36,8 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppMetrics;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppState;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttempt;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.AbstractYarnScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
-
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.FSAppAttempt;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.FairScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.FairSchedulerConfiguration;
@@ -149,13 +150,18 @@ public class TestRMWebAppFairScheduler {
       i++;
     }
 
-    return new RMContextImpl(null, null, null, null,
+    RMContextImpl rmContext =  new RMContextImpl(null, null, null, null,
         null, null, null, null, null, null) {
       @Override
       public ConcurrentMap<ApplicationId, RMApp> getRMApps() {
         return applicationsMaps;
       }
+      @Override
+      public ResourceScheduler getScheduler() {
+        return mock(AbstractYarnScheduler.class);
+      }
     };
+    return rmContext;
   }
 
   private static ResourceManager mockRm(RMContext rmContext) throws

http://git-wip-us.apache.org/repos/asf/hadoop/blob/eaccaba3/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesApps.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesApps.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesApps.java
index 705fd31..c60a584 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesApps.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesApps.java
@@ -1314,8 +1314,7 @@ public class TestRMWebServicesApps extends JerseyTestBase {
   public void verifyAppInfo(JSONObject info, RMApp app) throws JSONException,
       Exception {
 
-    // 28 because trackingUrl not assigned yet
-    assertEquals("incorrect number of elements", 26, info.length());
+    assertEquals("incorrect number of elements", 27, info.length());
 
     verifyAppInfoGeneric(app, info.getString("id"), info.getString("user"),
         info.getString("name"), info.getString("applicationType"),


[21/50] [abbrv] hadoop git commit: YARN-3255. RM, NM, JobHistoryServer, and WebAppProxyServer's main() should support generic options. Contributed by Konstantin Shvachko.

Posted by zh...@apache.org.
YARN-3255. RM, NM, JobHistoryServer, and WebAppProxyServer's main() should support generic options. Contributed by Konstantin Shvachko.


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

Branch: refs/heads/HDFS-7285
Commit: cc024460e00f99a5a5254ab7220d2efdf4797bac
Parents: 3d90a92
Author: Konstantin V Shvachko <sh...@apache.org>
Authored: Thu Feb 26 17:12:19 2015 -0800
Committer: Zhe Zhang <zh...@cloudera.com>
Committed: Mon Mar 2 09:13:53 2015 -0800

----------------------------------------------------------------------
 .../java/org/apache/hadoop/mapreduce/v2/hs/JobHistoryServer.java | 2 ++
 hadoop-yarn-project/CHANGES.txt                                  | 3 +++
 .../org/apache/hadoop/yarn/server/nodemanager/NodeManager.java   | 4 +++-
 .../hadoop/yarn/server/resourcemanager/ResourceManager.java      | 3 +++
 .../apache/hadoop/yarn/server/webproxy/WebAppProxyServer.java    | 2 ++
 5 files changed, 13 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/cc024460/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/JobHistoryServer.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/JobHistoryServer.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/JobHistoryServer.java
index 6d58040..252ac55 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/JobHistoryServer.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/JobHistoryServer.java
@@ -37,6 +37,7 @@ import org.apache.hadoop.security.SecurityUtil;
 import org.apache.hadoop.service.AbstractService;
 import org.apache.hadoop.service.CompositeService;
 import org.apache.hadoop.util.ExitUtil;
+import org.apache.hadoop.util.GenericOptionsParser;
 import org.apache.hadoop.util.ShutdownHookManager;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.yarn.YarnUncaughtExceptionHandler;
@@ -216,6 +217,7 @@ public class JobHistoryServer extends CompositeService {
           new CompositeServiceShutdownHook(jobHistoryServer),
           SHUTDOWN_HOOK_PRIORITY);
       YarnConfiguration conf = new YarnConfiguration(new JobConf());
+      new GenericOptionsParser(conf, args);
       jobHistoryServer.init(conf);
       jobHistoryServer.start();
     } catch (Throwable t) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cc024460/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index a635592..40f187b 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -330,6 +330,9 @@ Release 2.7.0 - UNRELEASED
     YARN-3217. Remove httpclient dependency from hadoop-yarn-server-web-proxy.
     (Brahma Reddy Battula via ozawa).
 
+    YARN-3255. RM, NM, JobHistoryServer, and WebAppProxyServer's main()
+    should support generic options. (shv)
+
   OPTIMIZATIONS
 
     YARN-2990. FairScheduler's delay-scheduling always waits for node-local and 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cc024460/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeManager.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/NodeManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeManager.java
index 7584138..a4be120 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeManager.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeManager.java
@@ -37,6 +37,7 @@ import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
 import org.apache.hadoop.security.Credentials;
 import org.apache.hadoop.security.SecurityUtil;
 import org.apache.hadoop.service.CompositeService;
+import org.apache.hadoop.util.GenericOptionsParser;
 import org.apache.hadoop.util.NodeHealthScriptRunner;
 import org.apache.hadoop.util.ReflectionUtils;
 import org.apache.hadoop.util.ShutdownHookManager;
@@ -523,11 +524,12 @@ public class NodeManager extends CompositeService
     return this.context;
   }
 
-  public static void main(String[] args) {
+  public static void main(String[] args) throws IOException {
     Thread.setDefaultUncaughtExceptionHandler(new YarnUncaughtExceptionHandler());
     StringUtils.startupShutdownMessage(NodeManager.class, args, LOG);
     NodeManager nodeManager = new NodeManager();
     Configuration conf = new YarnConfiguration();
+    new GenericOptionsParser(conf, args);
     nodeManager.initAndStartNodeManager(conf, false);
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cc024460/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 a93372a..8bd8e21 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
@@ -38,6 +38,7 @@ import org.apache.hadoop.service.AbstractService;
 import org.apache.hadoop.service.CompositeService;
 import org.apache.hadoop.service.Service;
 import org.apache.hadoop.util.ExitUtil;
+import org.apache.hadoop.util.GenericOptionsParser;
 import org.apache.hadoop.util.ReflectionUtils;
 import org.apache.hadoop.util.ShutdownHookManager;
 import org.apache.hadoop.util.StringUtils;
@@ -1193,6 +1194,8 @@ public class ResourceManager extends CompositeService implements Recoverable {
     StringUtils.startupShutdownMessage(ResourceManager.class, argv, LOG);
     try {
       Configuration conf = new YarnConfiguration();
+      GenericOptionsParser hParser = new GenericOptionsParser(conf, argv);
+      argv = hParser.getRemainingArgs();
       // If -format-state-store, then delete RMStateStore; else startup normally
       if (argv.length == 1 && argv[0].equals("-format-state-store")) {
         deleteRMStateStore(conf);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cc024460/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-web-proxy/src/main/java/org/apache/hadoop/yarn/server/webproxy/WebAppProxyServer.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-web-proxy/src/main/java/org/apache/hadoop/yarn/server/webproxy/WebAppProxyServer.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-web-proxy/src/main/java/org/apache/hadoop/yarn/server/webproxy/WebAppProxyServer.java
index 881e2e7..b1c563c 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-web-proxy/src/main/java/org/apache/hadoop/yarn/server/webproxy/WebAppProxyServer.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-web-proxy/src/main/java/org/apache/hadoop/yarn/server/webproxy/WebAppProxyServer.java
@@ -25,6 +25,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.security.SecurityUtil;
 import org.apache.hadoop.service.CompositeService;
 import org.apache.hadoop.util.ExitUtil;
+import org.apache.hadoop.util.GenericOptionsParser;
 import org.apache.hadoop.util.ShutdownHookManager;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.yarn.YarnUncaughtExceptionHandler;
@@ -89,6 +90,7 @@ public class WebAppProxyServer extends CompositeService {
     StringUtils.startupShutdownMessage(WebAppProxyServer.class, args, LOG);
     try {
       YarnConfiguration configuration = new YarnConfiguration();
+      new GenericOptionsParser(configuration, args);
       WebAppProxyServer proxyServer = startServer(configuration);
       proxyServer.proxy.join();
     } catch (Throwable t) {


[23/50] [abbrv] hadoop git commit: HDFS-7308. Change the packet chunk size computation in DFSOutputStream in order to enforce packet size <= 64kB. Contributed by Takuya Fukudome

Posted by zh...@apache.org.
HDFS-7308. Change the packet chunk size computation in DFSOutputStream in order to enforce packet size <= 64kB.  Contributed by Takuya Fukudome


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

Branch: refs/heads/HDFS-7285
Commit: 6eb3424388a2beeb290dfd670b5f6cf6b720080f
Parents: 54bcb5f
Author: Tsz-Wo Nicholas Sze <sz...@hortonworks.com>
Authored: Fri Feb 27 23:45:37 2015 +0800
Committer: Zhe Zhang <zh...@cloudera.com>
Committed: Mon Mar 2 09:13:53 2015 -0800

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |  3 ++
 .../org/apache/hadoop/hdfs/DFSOutputStream.java |  3 +-
 .../apache/hadoop/hdfs/TestDFSOutputStream.java | 31 ++++++++++++++++++++
 3 files changed, 36 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/6eb34243/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 8556afd..b2422d6 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -682,6 +682,9 @@ Release 2.7.0 - UNRELEASED
     HDFS-7819. Log WARN message for the blocks which are not in Block ID based
     layout (Rakesh R via Colin P. McCabe)
 
+    HDFS-7308. Change the packet chunk size computation in DFSOutputStream in
+    order to enforce packet size <= 64kB.  (Takuya Fukudome via szetszwo)
+
   OPTIMIZATIONS
 
     HDFS-7454. Reduce memory footprint for AclEntries in NameNode.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6eb34243/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java
index 9d7dca9..b3e8c97 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java
@@ -1851,8 +1851,9 @@ public class DFSOutputStream extends FSOutputSummer
   }
 
   private void computePacketChunkSize(int psize, int csize) {
+    final int bodySize = psize - PacketHeader.PKT_MAX_HEADER_LEN;
     final int chunkSize = csize + getChecksumSize();
-    chunksPerPacket = Math.max(psize/chunkSize, 1);
+    chunksPerPacket = Math.max(bodySize/chunkSize, 1);
     packetSize = chunkSize*chunksPerPacket;
     if (DFSClient.LOG.isDebugEnabled()) {
       DFSClient.LOG.debug("computePacketChunkSize: src=" + src +

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6eb34243/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSOutputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSOutputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSOutputStream.java
index 678a3b8..7269e39 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSOutputStream.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSOutputStream.java
@@ -18,6 +18,8 @@
 package org.apache.hadoop.hdfs;
 
 import java.io.IOException;
+import java.lang.reflect.Field;
+import java.lang.reflect.Method;
 import java.util.concurrent.atomic.AtomicReference;
 
 import org.apache.hadoop.conf.Configuration;
@@ -66,6 +68,35 @@ public class TestDFSOutputStream {
     dos.close();
   }
 
+  /**
+   * The computePacketChunkSize() method of DFSOutputStream should set the actual
+   * packet size < 64kB. See HDFS-7308 for details.
+   */
+  @Test
+  public void testComputePacketChunkSize()
+      throws Exception {
+    DistributedFileSystem fs = cluster.getFileSystem();
+    FSDataOutputStream os = fs.create(new Path("/test"));
+    DFSOutputStream dos = (DFSOutputStream) Whitebox.getInternalState(os,
+        "wrappedStream");
+
+    final int packetSize = 64*1024;
+    final int bytesPerChecksum = 512;
+
+    Method method = dos.getClass().getDeclaredMethod("computePacketChunkSize",
+        int.class, int.class);
+    method.setAccessible(true);
+    method.invoke(dos, packetSize, bytesPerChecksum);
+
+    Field field = dos.getClass().getDeclaredField("packetSize");
+    field.setAccessible(true);
+
+    Assert.assertTrue((Integer) field.get(dos) + 33 < packetSize);
+    // If PKT_MAX_HEADER_LEN is 257, actual packet size come to over 64KB
+    // without a fix on HDFS-7308.
+    Assert.assertTrue((Integer) field.get(dos) + 257 < packetSize);
+  }
+
   @AfterClass
   public static void tearDown() {
     cluster.shutdown();


[18/50] [abbrv] hadoop git commit: HADOOP-9922. hadoop windows native build will fail in 32 bit machine. Contributed by Kiran Kumar M R.

Posted by zh...@apache.org.
HADOOP-9922. hadoop windows native build will fail in 32 bit machine. Contributed by Kiran Kumar M R.


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

Branch: refs/heads/HDFS-7285
Commit: 9b0eda11cfa0539ee7b7bef52427342a96e75ec9
Parents: da85e17
Author: cnauroth <cn...@apache.org>
Authored: Thu Feb 26 12:41:33 2015 -0800
Committer: Zhe Zhang <zh...@cloudera.com>
Committed: Mon Mar 2 09:13:52 2015 -0800

----------------------------------------------------------------------
 hadoop-common-project/hadoop-common/CHANGES.txt |  3 +
 .../hadoop-common/src/main/native/native.sln    |  8 +--
 .../src/main/native/native.vcxproj              | 40 ++++++++++++
 .../src/main/winutils/include/winutils.h        |  5 +-
 .../src/main/winutils/libwinutils.c             |  2 +-
 .../src/main/winutils/libwinutils.vcxproj       | 64 +++++++++++++++++++-
 .../hadoop-common/src/main/winutils/service.c   |  8 +--
 .../hadoop-common/src/main/winutils/task.c      |  2 +-
 .../src/main/winutils/winutils.sln              | 10 +++
 .../src/main/winutils/winutils.vcxproj          | 61 ++++++++++++++++++-
 10 files changed, 189 insertions(+), 14 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/9b0eda11/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 ca27463..1d9a6d4 100644
--- a/hadoop-common-project/hadoop-common/CHANGES.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES.txt
@@ -1010,6 +1010,9 @@ Release 2.7.0 - UNRELEASED
     HADOOP-11629. WASB filesystem should not start BandwidthGaugeUpdater if
     fs.azure.skip.metrics set to true. (Shanyu Zhao via cnauroth)
 
+    HADOOP-9922. hadoop windows native build will fail in 32 bit machine.
+    (Kiran Kumar M R via cnauroth)
+
 Release 2.6.1 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9b0eda11/hadoop-common-project/hadoop-common/src/main/native/native.sln
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/native/native.sln b/hadoop-common-project/hadoop-common/src/main/native/native.sln
index 40a7821..54bc17e 100644
--- a/hadoop-common-project/hadoop-common/src/main/native/native.sln
+++ b/hadoop-common-project/hadoop-common/src/main/native/native.sln
@@ -31,14 +31,14 @@ Global
 	GlobalSection(ProjectConfigurationPlatforms) = postSolution
 		{4C0C12D2-3CB0-47F8-BCD0-55BD5732DFA7}.Debug|Mixed Platforms.ActiveCfg = Release|x64
 		{4C0C12D2-3CB0-47F8-BCD0-55BD5732DFA7}.Debug|Mixed Platforms.Build.0 = Release|x64
-		{4C0C12D2-3CB0-47F8-BCD0-55BD5732DFA7}.Debug|Win32.ActiveCfg = Release|x64
-		{4C0C12D2-3CB0-47F8-BCD0-55BD5732DFA7}.Debug|Win32.Build.0 = Release|x64
+		{4C0C12D2-3CB0-47F8-BCD0-55BD5732DFA7}.Debug|Win32.ActiveCfg = Release|Win32
+		{4C0C12D2-3CB0-47F8-BCD0-55BD5732DFA7}.Debug|Win32.Build.0 = Release|Win32
 		{4C0C12D2-3CB0-47F8-BCD0-55BD5732DFA7}.Debug|x64.ActiveCfg = Release|x64
 		{4C0C12D2-3CB0-47F8-BCD0-55BD5732DFA7}.Debug|x64.Build.0 = Release|x64
 		{4C0C12D2-3CB0-47F8-BCD0-55BD5732DFA7}.Release|Mixed Platforms.ActiveCfg = Release|x64
 		{4C0C12D2-3CB0-47F8-BCD0-55BD5732DFA7}.Release|Mixed Platforms.Build.0 = Release|x64
-		{4C0C12D2-3CB0-47F8-BCD0-55BD5732DFA7}.Release|Win32.ActiveCfg = Release|x64
-		{4C0C12D2-3CB0-47F8-BCD0-55BD5732DFA7}.Release|Win32.Build.0 = Release|x64
+		{4C0C12D2-3CB0-47F8-BCD0-55BD5732DFA7}.Release|Win32.ActiveCfg = Release|Win32
+		{4C0C12D2-3CB0-47F8-BCD0-55BD5732DFA7}.Release|Win32.Build.0 = Release|Win32
 		{4C0C12D2-3CB0-47F8-BCD0-55BD5732DFA7}.Release|x64.ActiveCfg = Release|x64
 		{4C0C12D2-3CB0-47F8-BCD0-55BD5732DFA7}.Release|x64.Build.0 = Release|x64
 	EndGlobalSection

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9b0eda11/hadoop-common-project/hadoop-common/src/main/native/native.vcxproj
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/native/native.vcxproj b/hadoop-common-project/hadoop-common/src/main/native/native.vcxproj
index 2d60e56..0912c6a 100644
--- a/hadoop-common-project/hadoop-common/src/main/native/native.vcxproj
+++ b/hadoop-common-project/hadoop-common/src/main/native/native.vcxproj
@@ -19,6 +19,10 @@
 
 <Project DefaultTargets="CheckRequireSnappy;Build" ToolsVersion="4.0" xmlns="http://schemas.microsoft.com/developer/msbuild/2003">
   <ItemGroup Label="ProjectConfigurations">
+    <ProjectConfiguration Include="Release|Win32">
+      <Configuration>Release</Configuration>
+      <Platform>Win32</Platform>
+    </ProjectConfiguration>
     <ProjectConfiguration Include="Release|x64">
       <Configuration>Release</Configuration>
       <Platform>x64</Platform>
@@ -30,6 +34,12 @@
     <RootNamespace>native</RootNamespace>
   </PropertyGroup>
   <Import Project="$(VCTargetsPath)\Microsoft.Cpp.Default.props" />
+  <PropertyGroup Condition="'$(Configuration)|$(Platform)'=='Release|Win32'" Label="Configuration">
+    <ConfigurationType>DynamicLibrary</ConfigurationType>
+    <UseDebugLibraries>false</UseDebugLibraries>
+    <WholeProgramOptimization>true</WholeProgramOptimization>
+    <CharacterSet>Unicode</CharacterSet>
+  </PropertyGroup>
   <PropertyGroup Condition="'$(Configuration)|$(Platform)'=='Release|x64'" Label="Configuration">
     <ConfigurationType>DynamicLibrary</ConfigurationType>
     <UseDebugLibraries>false</UseDebugLibraries>
@@ -39,10 +49,19 @@
   <Import Project="$(VCTargetsPath)\Microsoft.Cpp.props" />
   <ImportGroup Label="ExtensionSettings">
   </ImportGroup>
+  <ImportGroup Condition="'$(Configuration)|$(Platform)'=='Release|Win32'" Label="PropertySheets">
+    <Import Project="$(UserRootDir)\Microsoft.Cpp.$(Platform).user.props" Condition="exists('$(UserRootDir)\Microsoft.Cpp.$(Platform).user.props')" Label="LocalAppDataPlatform" />
+  </ImportGroup>
   <ImportGroup Condition="'$(Configuration)|$(Platform)'=='Release|x64'" Label="PropertySheets">
     <Import Project="$(UserRootDir)\Microsoft.Cpp.$(Platform).user.props" Condition="exists('$(UserRootDir)\Microsoft.Cpp.$(Platform).user.props')" Label="LocalAppDataPlatform" />
   </ImportGroup>
   <PropertyGroup Label="UserMacros" />
+  <PropertyGroup Condition="'$(Configuration)|$(Platform)'=='Release|Win32'">
+    <LinkIncremental>false</LinkIncremental>
+    <OutDir>..\..\..\target\bin\</OutDir>
+    <IntDir>..\..\..\target\native\$(Configuration)\</IntDir>
+    <TargetName>hadoop</TargetName>
+  </PropertyGroup>
   <PropertyGroup Condition="'$(Configuration)|$(Platform)'=='Release|x64'">
     <LinkIncremental>false</LinkIncremental>
     <OutDir>..\..\..\target\bin\</OutDir>
@@ -65,6 +84,27 @@
       Text="Required snappy library could not be found.  SnappyLibrary=$(SnappyLibrary), SnappyInclude=$(SnappyInclude), CustomSnappyLib=$(CustomSnappyLib), CustomSnappyInclude=$(CustomSnappyInclude), CustomSnappyPrefix=$(CustomSnappyPrefix)"
       Condition="'$(RequireSnappy)' == 'true' And '$(SnappyEnabled)' != 'true'" />
   </Target>
+  <ItemDefinitionGroup Condition="'$(Configuration)|$(Platform)'=='Release|Win32'">
+    <ClCompile>
+      <WarningLevel>Level3</WarningLevel>
+      <PrecompiledHeader>NotUsing</PrecompiledHeader>
+      <Optimization>MaxSpeed</Optimization>
+      <FunctionLevelLinking>true</FunctionLevelLinking>
+      <IntrinsicFunctions>true</IntrinsicFunctions>
+      <PreprocessorDefinitions>WIN32;NDEBUG;_WINDOWS;_USRDLL;NATIVE_EXPORTS;%(PreprocessorDefinitions)</PreprocessorDefinitions>
+      <AdditionalIncludeDirectories>..\winutils\include;..\..\..\target\native\javah;%JAVA_HOME%\include;%JAVA_HOME%\include\win32;.\src;%(AdditionalIncludeDirectories)</AdditionalIncludeDirectories>
+      <CompileAs>CompileAsC</CompileAs>
+      <DisableSpecificWarnings>4244</DisableSpecificWarnings>
+    </ClCompile>
+    <Link>
+      <SubSystem>Windows</SubSystem>
+      <GenerateDebugInformation>true</GenerateDebugInformation>
+      <EnableCOMDATFolding>true</EnableCOMDATFolding>
+      <OptimizeReferences>true</OptimizeReferences>
+      <AdditionalDependencies>Ws2_32.lib;libwinutils.lib;%(AdditionalDependencies)</AdditionalDependencies>
+      <AdditionalLibraryDirectories>..\..\..\target\bin;%(AdditionalLibraryDirectories)</AdditionalLibraryDirectories>
+    </Link>
+  </ItemDefinitionGroup>
   <ItemDefinitionGroup Condition="'$(Configuration)|$(Platform)'=='Release|x64'">
     <ClCompile>
       <WarningLevel>Level3</WarningLevel>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9b0eda11/hadoop-common-project/hadoop-common/src/main/winutils/include/winutils.h
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/winutils/include/winutils.h b/hadoop-common-project/hadoop-common/src/main/winutils/include/winutils.h
index 0ac9adb..f72802c 100644
--- a/hadoop-common-project/hadoop-common/src/main/winutils/include/winutils.h
+++ b/hadoop-common-project/hadoop-common/src/main/winutils/include/winutils.h
@@ -248,8 +248,9 @@ DWORD BuildServiceSecurityDescriptor(
   __out PSECURITY_DESCRIPTOR*         pSD);
 
 DWORD AddNodeManagerAndUserACEsToObject(
-  __in HANDLE hProcess,
-  __in LPWSTR user);
+  __in HANDLE hObject,
+  __in LPWSTR user,
+  __in ACCESS_MASK accessMask);
 
 
 DWORD GetSecureJobObjectName(

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9b0eda11/hadoop-common-project/hadoop-common/src/main/winutils/libwinutils.c
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/winutils/libwinutils.c b/hadoop-common-project/hadoop-common/src/main/winutils/libwinutils.c
index 5e775df..98fe3ab 100644
--- a/hadoop-common-project/hadoop-common/src/main/winutils/libwinutils.c
+++ b/hadoop-common-project/hadoop-common/src/main/winutils/libwinutils.c
@@ -2788,7 +2788,7 @@ DWORD BuildServiceSecurityDescriptor(
   __out PSECURITY_DESCRIPTOR*         pSD) {
 
   DWORD                 dwError = ERROR_SUCCESS;
-  int                   crt  = 0;
+  unsigned int          crt  = 0;
   int                   len = 0;
   EXPLICIT_ACCESS*      eas = NULL;
   LPWSTR                lpszSD = NULL;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9b0eda11/hadoop-common-project/hadoop-common/src/main/winutils/libwinutils.vcxproj
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/winutils/libwinutils.vcxproj b/hadoop-common-project/hadoop-common/src/main/winutils/libwinutils.vcxproj
index 37b7f31..3c958e0 100644
--- a/hadoop-common-project/hadoop-common/src/main/winutils/libwinutils.vcxproj
+++ b/hadoop-common-project/hadoop-common/src/main/winutils/libwinutils.vcxproj
@@ -19,10 +19,18 @@
 
 <Project DefaultTargets="Build" ToolsVersion="4.0" xmlns="http://schemas.microsoft.com/developer/msbuild/2003">
   <ItemGroup Label="ProjectConfigurations">
+    <ProjectConfiguration Include="Debug|Win32">
+      <Configuration>Debug</Configuration>
+      <Platform>Win32</Platform>
+    </ProjectConfiguration>
     <ProjectConfiguration Include="Debug|x64">
       <Configuration>Debug</Configuration>
       <Platform>x64</Platform>
     </ProjectConfiguration>
+    <ProjectConfiguration Include="Release|Win32">
+      <Configuration>Release</Configuration>
+      <Platform>Win32</Platform>
+    </ProjectConfiguration>
     <ProjectConfiguration Include="Release|x64">
       <Configuration>Release</Configuration>
       <Platform>x64</Platform>
@@ -39,21 +47,38 @@
     <UseDebugLibraries>true</UseDebugLibraries>
     <CharacterSet>Unicode</CharacterSet>
   </PropertyGroup>
+  <PropertyGroup Condition="'$(Configuration)|$(Platform)'=='Debug|Win32'" Label="Configuration">
+    <ConfigurationType>StaticLibrary</ConfigurationType>
+    <UseDebugLibraries>true</UseDebugLibraries>
+    <CharacterSet>Unicode</CharacterSet>
+  </PropertyGroup>
   <PropertyGroup Condition="'$(Configuration)|$(Platform)'=='Release|x64'" Label="Configuration">
     <ConfigurationType>StaticLibrary</ConfigurationType>
     <UseDebugLibraries>false</UseDebugLibraries>
     <WholeProgramOptimization>true</WholeProgramOptimization>
     <CharacterSet>Unicode</CharacterSet>
   </PropertyGroup>
+  <PropertyGroup Condition="'$(Configuration)|$(Platform)'=='Release|Win32'" Label="Configuration">
+    <ConfigurationType>StaticLibrary</ConfigurationType>
+    <UseDebugLibraries>false</UseDebugLibraries>
+    <WholeProgramOptimization>true</WholeProgramOptimization>
+    <CharacterSet>Unicode</CharacterSet>
+  </PropertyGroup>
   <Import Project="$(VCTargetsPath)\Microsoft.Cpp.props" />
   <ImportGroup Label="ExtensionSettings">
   </ImportGroup>
   <ImportGroup Condition="'$(Configuration)|$(Platform)'=='Debug|x64'" Label="PropertySheets">
     <Import Project="$(UserRootDir)\Microsoft.Cpp.$(Platform).user.props" Condition="exists('$(UserRootDir)\Microsoft.Cpp.$(Platform).user.props')" Label="LocalAppDataPlatform" />
   </ImportGroup>
+  <ImportGroup Condition="'$(Configuration)|$(Platform)'=='Debug|Win32'" Label="PropertySheets">
+    <Import Project="$(UserRootDir)\Microsoft.Cpp.$(Platform).user.props" Condition="exists('$(UserRootDir)\Microsoft.Cpp.$(Platform).user.props')" Label="LocalAppDataPlatform" />
+  </ImportGroup>
   <ImportGroup Condition="'$(Configuration)|$(Platform)'=='Release|x64'" Label="PropertySheets">
     <Import Project="$(UserRootDir)\Microsoft.Cpp.$(Platform).user.props" Condition="exists('$(UserRootDir)\Microsoft.Cpp.$(Platform).user.props')" Label="LocalAppDataPlatform" />
   </ImportGroup>
+  <ImportGroup Condition="'$(Configuration)|$(Platform)'=='Release|Win32'" Label="PropertySheets">
+    <Import Project="$(UserRootDir)\Microsoft.Cpp.$(Platform).user.props" Condition="exists('$(UserRootDir)\Microsoft.Cpp.$(Platform).user.props')" Label="LocalAppDataPlatform" />
+  </ImportGroup>
   <PropertyGroup Label="UserMacros" />
   <PropertyGroup>
     <IncludePath>include;$(IncludePath)</IncludePath>
@@ -61,9 +86,15 @@
   <PropertyGroup Condition="'$(Configuration)|$(Platform)'=='Debug|x64'">
     <LinkIncremental>true</LinkIncremental>
   </PropertyGroup>
+  <PropertyGroup Condition="'$(Configuration)|$(Platform)'=='Debug|Win32'">
+    <LinkIncremental>true</LinkIncremental>
+  </PropertyGroup>
   <PropertyGroup Condition="'$(Configuration)|$(Platform)'=='Release|x64'">
     <LinkIncremental>false</LinkIncremental>
   </PropertyGroup>
+  <PropertyGroup Condition="'$(Configuration)|$(Platform)'=='Release|Win32'">
+    <LinkIncremental>false</LinkIncremental>
+  </PropertyGroup>
   <ItemDefinitionGroup Condition="'$(Configuration)|$(Platform)'=='Debug|x64'">
     <ClCompile>
       <PrecompiledHeader>
@@ -77,6 +108,19 @@
       <GenerateDebugInformation>true</GenerateDebugInformation>
     </Link>
   </ItemDefinitionGroup>
+  <ItemDefinitionGroup Condition="'$(Configuration)|$(Platform)'=='Debug|Win32'">
+    <ClCompile>
+      <PrecompiledHeader>
+      </PrecompiledHeader>
+      <WarningLevel>Level4</WarningLevel>
+      <Optimization>Disabled</Optimization>
+      <PreprocessorDefinitions>WIN32;_DEBUG;_UNICODE;UNICODE;WSCE_CONFIG_DIR=$(WsceConfigDir);WSCE_CONFIG_FILE=$(WsceConfigFile);%(PreprocessorDefinitions)</PreprocessorDefinitions>
+    </ClCompile>
+    <Link>
+      <SubSystem>Console</SubSystem>
+      <GenerateDebugInformation>true</GenerateDebugInformation>
+    </Link>
+  </ItemDefinitionGroup>
   <ItemDefinitionGroup Condition="'$(Configuration)|$(Platform)'=='Release|x64'">
     <ClCompile>
       <WarningLevel>Level3</WarningLevel>
@@ -95,13 +139,31 @@
       <OptimizeReferences>true</OptimizeReferences>
     </Link>
   </ItemDefinitionGroup>
+  <ItemDefinitionGroup Condition="'$(Configuration)|$(Platform)'=='Release|Win32'">
+    <ClCompile>
+      <WarningLevel>Level3</WarningLevel>
+      <PrecompiledHeader>
+      </PrecompiledHeader>
+      <!-- <Optimization>MaxSpeed</Optimization> -->
+      <Optimization>Disabled</Optimization>
+      <FunctionLevelLinking>true</FunctionLevelLinking>
+      <IntrinsicFunctions>true</IntrinsicFunctions>
+      <PreprocessorDefinitions>WIN32;NDEBUG;_UNICODE;UNICODE;WSCE_CONFIG_DIR=$(WsceConfigDir);WSCE_CONFIG_FILE=$(WsceConfigFile);%(PreprocessorDefinitions)</PreprocessorDefinitions>
+    </ClCompile>
+    <Link>
+      <SubSystem>Console</SubSystem>
+      <GenerateDebugInformation>true</GenerateDebugInformation>
+      <EnableCOMDATFolding>true</EnableCOMDATFolding>
+      <OptimizeReferences>true</OptimizeReferences>
+    </Link>
+  </ItemDefinitionGroup>
   <ItemDefinitionGroup>
     <ClCompile>
       <AdditionalIncludeDirectories>$(IntermediateOutputPath)</AdditionalIncludeDirectories>
     </ClCompile>
     <Midl>
       <ApplicationConfigurationMode>true</ApplicationConfigurationMode>
-      <TargetEnvironment>X64</TargetEnvironment>
+      <TargetEnvironment>$(Platform)</TargetEnvironment>
       <OutputDirectory>$(IntermediateOutputPath)</OutputDirectory>
       <GenerateStublessProxies>true</GenerateStublessProxies>
       <ValidateAllParameters>true</ValidateAllParameters>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9b0eda11/hadoop-common-project/hadoop-common/src/main/winutils/service.c
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/winutils/service.c b/hadoop-common-project/hadoop-common/src/main/winutils/service.c
index a0f8a66..ba35003 100644
--- a/hadoop-common-project/hadoop-common/src/main/winutils/service.c
+++ b/hadoop-common-project/hadoop-common/src/main/winutils/service.c
@@ -182,7 +182,7 @@ BOOL IsSidInList(
   __in size_t cAllowedSids, 
   __in_ecount(cAllowedSids) PSID* allowedSids) {
 
-  int crtSid = 0;
+  size_t crtSid = 0;
   
   for (crtSid = 0; crtSid < cAllowedSids; ++crtSid) {
     if (EqualSid(trustee, allowedSids[crtSid])) {
@@ -206,7 +206,7 @@ DWORD ValidateConfigurationFile() {
   BOOL daclPresent = FALSE;
   BOOL daclDefaulted = FALSE;
   PACL pDacl = NULL;
-  int crt = 0, crtSid = 0;
+  unsigned int crt = 0, crtSid = 0;
   WELL_KNOWN_SID_TYPE allowedSidTypes[] = {
     WinLocalSystemSid,
     WinBuiltinAdministratorsSid};
@@ -350,7 +350,7 @@ DWORD InitLocalDirs() {
   DWORD     dwError = ERROR_SUCCESS;
   size_t    len = 0;
   LPCWSTR   value = NULL;
-  int       crt = 0;
+  size_t    crt = 0;
     
 
   dwError = GetConfigValue(
@@ -396,7 +396,7 @@ done:
 DWORD ValidateLocalPath(LPCWSTR lpszPath) {
   DWORD   dwError = ERROR_SUCCESS;
   int     compareResult = 0;
-  int     crt = 0;
+  unsigned int  crt = 0;
   int     cchLocalBuffer = 0;
   WCHAR   localBuffer[MAX_PATH+1];
   BOOLEAN nullFound = FALSE;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9b0eda11/hadoop-common-project/hadoop-common/src/main/winutils/task.c
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/winutils/task.c b/hadoop-common-project/hadoop-common/src/main/winutils/task.c
index bfdbd63..21b1893 100644
--- a/hadoop-common-project/hadoop-common/src/main/winutils/task.c
+++ b/hadoop-common-project/hadoop-common/src/main/winutils/task.c
@@ -129,7 +129,7 @@ DWORD BuildImpersonateSecurityDescriptor(__out PSECURITY_DESCRIPTOR* ppSD) {
   WCHAR** tokens = NULL;
   size_t len = 0;
   size_t count = 0;
-  int crt = 0;
+  size_t crt = 0;
   PSECURITY_DESCRIPTOR pSD = NULL;
 
   dwError = GetConfigValue(wsceConfigRelativePath, NM_WSCE_IMPERSONATE_ALLOWED, &len, &value); 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9b0eda11/hadoop-common-project/hadoop-common/src/main/winutils/winutils.sln
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/winutils/winutils.sln b/hadoop-common-project/hadoop-common/src/main/winutils/winutils.sln
index d2784b8..876e66d 100644
--- a/hadoop-common-project/hadoop-common/src/main/winutils/winutils.sln
+++ b/hadoop-common-project/hadoop-common/src/main/winutils/winutils.sln
@@ -26,16 +26,26 @@ Project("{8BC9CEB8-8B4A-11D0-8D11-00A0C91BC942}") = "libwinutils", "libwinutils.
 EndProject
 Global
 	GlobalSection(SolutionConfigurationPlatforms) = preSolution
+		Debug|Win32 = Debug|Win32
 		Debug|x64 = Debug|x64
+		Release|Win32 = Release|Win32
 		Release|x64 = Release|x64
 	EndGlobalSection
 	GlobalSection(ProjectConfigurationPlatforms) = postSolution
+		{D94B3BD7-39CC-47A0-AE9A-353FDE506F33}.Debug|Win32.ActiveCfg = Release|Win32
+		{D94B3BD7-39CC-47A0-AE9A-353FDE506F33}.Debug|Win32.Build.0 = Release|Win32
 		{D94B3BD7-39CC-47A0-AE9A-353FDE506F33}.Debug|x64.ActiveCfg = Release|x64
 		{D94B3BD7-39CC-47A0-AE9A-353FDE506F33}.Debug|x64.Build.0 = Release|x64
+		{D94B3BD7-39CC-47A0-AE9A-353FDE506F33}.Release|Win32.ActiveCfg = Release|Win32
+		{D94B3BD7-39CC-47A0-AE9A-353FDE506F33}.Release|Win32.Build.0 = Release|Win32
 		{D94B3BD7-39CC-47A0-AE9A-353FDE506F33}.Release|x64.ActiveCfg = Release|x64
 		{D94B3BD7-39CC-47A0-AE9A-353FDE506F33}.Release|x64.Build.0 = Release|x64
+		{12131AA7-902E-4A6D-9CE3-043261D22A12}.Debug|Win32.ActiveCfg = Release|Win32
+		{12131AA7-902E-4A6D-9CE3-043261D22A12}.Debug|Win32.Build.0 = Release|Win32
 		{12131AA7-902E-4A6D-9CE3-043261D22A12}.Debug|x64.ActiveCfg = Release|x64
 		{12131AA7-902E-4A6D-9CE3-043261D22A12}.Debug|x64.Build.0 = Release|x64
+		{12131AA7-902E-4A6D-9CE3-043261D22A12}.Release|Win32.ActiveCfg = Release|Win32
+		{12131AA7-902E-4A6D-9CE3-043261D22A12}.Release|Win32.Build.0 = Release|Win32
 		{12131AA7-902E-4A6D-9CE3-043261D22A12}.Release|x64.ActiveCfg = Release|x64
 		{12131AA7-902E-4A6D-9CE3-043261D22A12}.Release|x64.Build.0 = Release|x64
 	EndGlobalSection

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9b0eda11/hadoop-common-project/hadoop-common/src/main/winutils/winutils.vcxproj
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/winutils/winutils.vcxproj b/hadoop-common-project/hadoop-common/src/main/winutils/winutils.vcxproj
index d736084..9ecba0a 100644
--- a/hadoop-common-project/hadoop-common/src/main/winutils/winutils.vcxproj
+++ b/hadoop-common-project/hadoop-common/src/main/winutils/winutils.vcxproj
@@ -19,10 +19,18 @@
 
 <Project DefaultTargets="Build" ToolsVersion="4.0" xmlns="http://schemas.microsoft.com/developer/msbuild/2003">
   <ItemGroup Label="ProjectConfigurations">
+    <ProjectConfiguration Include="Debug|Win32">
+      <Configuration>Debug</Configuration>
+      <Platform>Win32</Platform>
+    </ProjectConfiguration>
     <ProjectConfiguration Include="Debug|x64">
       <Configuration>Debug</Configuration>
       <Platform>x64</Platform>
     </ProjectConfiguration>
+    <ProjectConfiguration Include="Release|Win32">
+      <Configuration>Release</Configuration>
+      <Platform>Win32</Platform>
+    </ProjectConfiguration>
     <ProjectConfiguration Include="Release|x64">
       <Configuration>Release</Configuration>
       <Platform>x64</Platform>
@@ -39,6 +47,11 @@
     <UseDebugLibraries>true</UseDebugLibraries>
     <CharacterSet>Unicode</CharacterSet>
   </PropertyGroup>
+  <PropertyGroup Condition="'$(Configuration)|$(Platform)'=='Debug|Win32'" Label="Configuration">
+    <ConfigurationType>Application</ConfigurationType>
+    <UseDebugLibraries>true</UseDebugLibraries>
+    <CharacterSet>Unicode</CharacterSet>
+  </PropertyGroup>
   <PropertyGroup Condition="'$(Configuration)|$(Platform)'=='Release|x64'" Label="Configuration">
     <ConfigurationType>Application</ConfigurationType>
     <UseDebugLibraries>false</UseDebugLibraries>
@@ -46,14 +59,26 @@
     <CharacterSet>Unicode</CharacterSet>
   </PropertyGroup>
   <Import Project="$(VCTargetsPath)\Microsoft.Cpp.props" />
+  <PropertyGroup Condition="'$(Configuration)|$(Platform)'=='Release|Win32'" Label="Configuration">
+    <ConfigurationType>Application</ConfigurationType>
+    <UseDebugLibraries>false</UseDebugLibraries>
+    <WholeProgramOptimization>true</WholeProgramOptimization>
+    <CharacterSet>Unicode</CharacterSet>
+  </PropertyGroup>
   <ImportGroup Label="ExtensionSettings">
   </ImportGroup>
   <ImportGroup Condition="'$(Configuration)|$(Platform)'=='Debug|x64'" Label="PropertySheets">
     <Import Project="$(UserRootDir)\Microsoft.Cpp.$(Platform).user.props" Condition="exists('$(UserRootDir)\Microsoft.Cpp.$(Platform).user.props')" Label="LocalAppDataPlatform" />
   </ImportGroup>
+  <ImportGroup Condition="'$(Configuration)|$(Platform)'=='Debug|Win32'" Label="PropertySheets">
+    <Import Project="$(UserRootDir)\Microsoft.Cpp.$(Platform).user.props" Condition="exists('$(UserRootDir)\Microsoft.Cpp.$(Platform).user.props')" Label="LocalAppDataPlatform" />
+  </ImportGroup>
   <ImportGroup Condition="'$(Configuration)|$(Platform)'=='Release|x64'" Label="PropertySheets">
     <Import Project="$(UserRootDir)\Microsoft.Cpp.$(Platform).user.props" Condition="exists('$(UserRootDir)\Microsoft.Cpp.$(Platform).user.props')" Label="LocalAppDataPlatform" />
   </ImportGroup>
+  <ImportGroup Condition="'$(Configuration)|$(Platform)'=='Release|Win32'" Label="PropertySheets">
+    <Import Project="$(UserRootDir)\Microsoft.Cpp.$(Platform).user.props" Condition="exists('$(UserRootDir)\Microsoft.Cpp.$(Platform).user.props')" Label="LocalAppDataPlatform" />
+  </ImportGroup>
   <PropertyGroup Label="UserMacros" />
   <PropertyGroup>
     <IncludePath>include;$(IncludePath)</IncludePath>
@@ -61,6 +86,9 @@
   <PropertyGroup Condition="'$(Configuration)|$(Platform)'=='Release|x64'">
     <LinkIncremental>false</LinkIncremental>
   </PropertyGroup>
+  <PropertyGroup Condition="'$(Configuration)|$(Platform)'=='Release|Win32'">
+    <LinkIncremental>false</LinkIncremental>
+  </PropertyGroup>
   <ItemDefinitionGroup Condition="'$(Configuration)|$(Platform)'=='Debug|x64'">
     <ClCompile>
       <PrecompiledHeader>
@@ -74,6 +102,19 @@
       <GenerateDebugInformation>true</GenerateDebugInformation>
     </Link>
   </ItemDefinitionGroup>
+  <ItemDefinitionGroup Condition="'$(Configuration)|$(Platform)'=='Debug|Win32'">
+    <ClCompile>
+      <PrecompiledHeader>
+      </PrecompiledHeader>
+      <WarningLevel>Level4</WarningLevel>
+      <Optimization>Disabled</Optimization>
+      <PreprocessorDefinitions>WIN32;_CONSOLE;_DEBUG;_UNICODE;UNICODE;WSCE_CONFIG_DIR=$(WsceConfigDir);WSCE_CONFIG_FILE=$(WsceConfigFile);%(PreprocessorDefinitions)</PreprocessorDefinitions>
+    </ClCompile>
+    <Link>
+      <SubSystem>Console</SubSystem>
+      <GenerateDebugInformation>true</GenerateDebugInformation>
+    </Link>
+  </ItemDefinitionGroup>
   <ItemDefinitionGroup Condition="'$(Configuration)|$(Platform)'=='Release|x64'">
     <ClCompile>
       <WarningLevel>Level3</WarningLevel>
@@ -92,6 +133,24 @@
       <OptimizeReferences>true</OptimizeReferences>
     </Link>
   </ItemDefinitionGroup>
+  <ItemDefinitionGroup Condition="'$(Configuration)|$(Platform)'=='Release|Win32'">
+    <ClCompile>
+      <WarningLevel>Level3</WarningLevel>
+      <PrecompiledHeader>
+      </PrecompiledHeader>
+      <!-- <Optimization>MaxSpeed</Optimization> -->
+      <Optimization>Disabled</Optimization>
+      <FunctionLevelLinking>true</FunctionLevelLinking>
+      <IntrinsicFunctions>true</IntrinsicFunctions>
+      <PreprocessorDefinitions>WIN32;_CONSOLE;NDEBUG;_UNICODE;UNICODE;WSCE_CONFIG_DIR=$(WsceConfigDir);WSCE_CONFIG_FILE=$(WsceConfigFile);%(PreprocessorDefinitions)</PreprocessorDefinitions>
+    </ClCompile>
+    <Link>
+      <SubSystem>Console</SubSystem>
+      <GenerateDebugInformation>true</GenerateDebugInformation>
+      <EnableCOMDATFolding>true</EnableCOMDATFolding>
+      <OptimizeReferences>true</OptimizeReferences>
+    </Link>
+  </ItemDefinitionGroup>
   <ItemDefinitionGroup>
     <ClCompile>
       <AdditionalIncludeDirectories>$(IntermediateOutputPath)</AdditionalIncludeDirectories>
@@ -103,7 +162,7 @@
     </CustomBuildStep>
     <Midl>
       <ApplicationConfigurationMode>true</ApplicationConfigurationMode>
-      <TargetEnvironment>X64</TargetEnvironment>
+      <TargetEnvironment>$(Platform)</TargetEnvironment>
       <OutputDirectory>$(IntermediateOutputPath)</OutputDirectory>
       <GenerateStublessProxies>true</GenerateStublessProxies>
       <ValidateAllParameters>true</ValidateAllParameters>


[25/50] [abbrv] hadoop git commit: YARN-2820. Retry in FileSystemRMStateStore when FS's operations fail due to IOException. Contributed by Zhihai Xu.

Posted by zh...@apache.org.
YARN-2820. Retry in FileSystemRMStateStore when FS's operations fail due to IOException. Contributed by Zhihai Xu.


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

Branch: refs/heads/HDFS-7285
Commit: 4e95f9870ed33fe3cd11e73981b24f0dc651bd77
Parents: 6eb3424
Author: Tsuyoshi Ozawa <oz...@apache.org>
Authored: Sat Feb 28 00:56:44 2015 +0900
Committer: Zhe Zhang <zh...@cloudera.com>
Committed: Mon Mar 2 09:13:53 2015 -0800

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                 |   3 +
 .../hadoop/yarn/conf/YarnConfiguration.java     |   9 +
 .../src/main/resources/yarn-default.xml         |  15 +
 .../recovery/FileSystemRMStateStore.java        | 303 ++++++++++++++-----
 .../recovery/TestFSRMStateStore.java            |   5 +
 5 files changed, 265 insertions(+), 70 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/4e95f987/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index 40f187b..38dd9fa 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -333,6 +333,9 @@ Release 2.7.0 - UNRELEASED
     YARN-3255. RM, NM, JobHistoryServer, and WebAppProxyServer's main()
     should support generic options. (shv)
 
+    YARN-2820. Retry in FileSystemRMStateStore when FS's operations fail 
+    due to IOException. (Zhihai Xu via ozawa)
+
   OPTIMIZATIONS
 
     YARN-2990. FairScheduler's delay-scheduling always waits for node-local and 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4e95f987/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 05c6cbf..ff06eea 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
@@ -508,6 +508,15 @@ public class YarnConfiguration extends Configuration {
   public static final String DEFAULT_FS_RM_STATE_STORE_RETRY_POLICY_SPEC =
       "2000, 500";
 
+  public static final String FS_RM_STATE_STORE_NUM_RETRIES =
+      RM_PREFIX + "fs.state-store.num-retries";
+  public static final int DEFAULT_FS_RM_STATE_STORE_NUM_RETRIES = 0;
+
+  public static final String FS_RM_STATE_STORE_RETRY_INTERVAL_MS =
+      RM_PREFIX + "fs.state-store.retry-interval-ms";
+  public static final long DEFAULT_FS_RM_STATE_STORE_RETRY_INTERVAL_MS =
+      1000L;
+
   public static final String RM_LEVELDB_STORE_PATH = RM_PREFIX
       + "leveldb-state-store.path";
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4e95f987/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 a7958a5..df730d5 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
@@ -420,6 +420,21 @@
   </property>
 
   <property>
+    <description>the number of retries to recover from IOException in
+    FileSystemRMStateStore.
+    </description>
+    <name>yarn.resourcemanager.fs.state-store.num-retries</name>
+    <value>0</value>
+  </property>
+
+  <property>
+    <description>Retry interval in milliseconds in FileSystemRMStateStore.
+    </description>
+    <name>yarn.resourcemanager.fs.state-store.retry-interval-ms</name>
+    <value>1000</value>
+  </property>
+
+  <property>
     <description>Local path where the RM state will be stored when using
     org.apache.hadoop.yarn.server.resourcemanager.recovery.LeveldbRMStateStore
     as the value for yarn.resourcemanager.store.class</description>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4e95f987/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/FileSystemRMStateStore.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/recovery/FileSystemRMStateStore.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/FileSystemRMStateStore.java
index 6e830a0..8147597 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/FileSystemRMStateStore.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/FileSystemRMStateStore.java
@@ -92,6 +92,8 @@ public class FileSystemRMStateStore extends RMStateStore {
   Path rmDTSecretManagerRoot;
   private Path rmAppRoot;
   private Path dtSequenceNumberPath = null;
+  private int fsNumRetries;
+  private long fsRetryInterval;
 
   @VisibleForTesting
   Path fsWorkingPath;
@@ -106,6 +108,12 @@ public class FileSystemRMStateStore extends RMStateStore {
     rmAppRoot = new Path(rootDirPath, RM_APP_ROOT);
     amrmTokenSecretManagerRoot =
         new Path(rootDirPath, AMRMTOKEN_SECRET_MANAGER_ROOT);
+    fsNumRetries =
+        conf.getInt(YarnConfiguration.FS_RM_STATE_STORE_NUM_RETRIES,
+            YarnConfiguration.DEFAULT_FS_RM_STATE_STORE_NUM_RETRIES);
+    fsRetryInterval =
+        conf.getLong(YarnConfiguration.FS_RM_STATE_STORE_RETRY_INTERVAL_MS,
+                YarnConfiguration.DEFAULT_FS_RM_STATE_STORE_RETRY_INTERVAL_MS);
   }
 
   @Override
@@ -121,14 +129,14 @@ public class FileSystemRMStateStore extends RMStateStore {
     conf.set("dfs.client.retry.policy.spec", retryPolicy);
 
     fs = fsWorkingPath.getFileSystem(conf);
-    fs.mkdirs(rmDTSecretManagerRoot);
-    fs.mkdirs(rmAppRoot);
-    fs.mkdirs(amrmTokenSecretManagerRoot);
+    mkdirsWithRetries(rmDTSecretManagerRoot);
+    mkdirsWithRetries(rmAppRoot);
+    mkdirsWithRetries(amrmTokenSecretManagerRoot);
   }
 
   @Override
   protected synchronized void closeInternal() throws Exception {
-    fs.close();
+    closeWithRetries();
   }
 
   @Override
@@ -139,9 +147,9 @@ public class FileSystemRMStateStore extends RMStateStore {
   @Override
   protected synchronized Version loadVersion() throws Exception {
     Path versionNodePath = getNodePath(rootDirPath, VERSION_NODE);
-    FileStatus status = getFileStatus(versionNodePath);
+    FileStatus status = getFileStatusWithRetries(versionNodePath);
     if (status != null) {
-      byte[] data = readFile(versionNodePath, status.getLen());
+      byte[] data = readFileWithRetries(versionNodePath, status.getLen());
       Version version =
           new VersionPBImpl(VersionProto.parseFrom(data));
       return version;
@@ -154,10 +162,10 @@ public class FileSystemRMStateStore extends RMStateStore {
     Path versionNodePath = getNodePath(rootDirPath, VERSION_NODE);
     byte[] data =
         ((VersionPBImpl) CURRENT_VERSION_INFO).getProto().toByteArray();
-    if (fs.exists(versionNodePath)) {
+    if (existsWithRetries(versionNodePath)) {
       updateFile(versionNodePath, data);
     } else {
-      writeFile(versionNodePath, data);
+      writeFileWithRetries(versionNodePath, data);
     }
   }
   
@@ -165,10 +173,10 @@ public class FileSystemRMStateStore extends RMStateStore {
   public synchronized long getAndIncrementEpoch() throws Exception {
     Path epochNodePath = getNodePath(rootDirPath, EPOCH_NODE);
     long currentEpoch = 0;
-    FileStatus status = getFileStatus(epochNodePath);
+    FileStatus status = getFileStatusWithRetries(epochNodePath);
     if (status != null) {
       // load current epoch
-      byte[] data = readFile(epochNodePath, status.getLen());
+      byte[] data = readFileWithRetries(epochNodePath, status.getLen());
       Epoch epoch = new EpochPBImpl(EpochProto.parseFrom(data));
       currentEpoch = epoch.getEpoch();
       // increment epoch and store it
@@ -179,7 +187,7 @@ public class FileSystemRMStateStore extends RMStateStore {
       // initialize epoch file with 1 for the next time.
       byte[] storeData = Epoch.newInstance(currentEpoch + 1).getProto()
           .toByteArray();
-      writeFile(epochNodePath, storeData);
+      writeFileWithRetries(epochNodePath, storeData);
     }
     return currentEpoch;
   }
@@ -201,12 +209,14 @@ public class FileSystemRMStateStore extends RMStateStore {
     checkAndResumeUpdateOperation(amrmTokenSecretManagerRoot);
     Path amrmTokenSecretManagerStateDataDir =
         new Path(amrmTokenSecretManagerRoot, AMRMTOKEN_SECRET_MANAGER_NODE);
-    FileStatus status = getFileStatus(amrmTokenSecretManagerStateDataDir);
+    FileStatus status = getFileStatusWithRetries(
+        amrmTokenSecretManagerStateDataDir);
     if (status == null) {
       return;
     }
     assert status.isFile();
-    byte[] data = readFile(amrmTokenSecretManagerStateDataDir, status.getLen());
+    byte[] data = readFileWithRetries(amrmTokenSecretManagerStateDataDir,
+            status.getLen());
     AMRMTokenSecretManagerStatePBImpl stateData =
         new AMRMTokenSecretManagerStatePBImpl(
           AMRMTokenSecretManagerStateProto.parseFrom(data));
@@ -220,16 +230,18 @@ public class FileSystemRMStateStore extends RMStateStore {
       List<ApplicationAttemptStateData> attempts =
           new ArrayList<ApplicationAttemptStateData>();
 
-      for (FileStatus appDir : fs.listStatus(rmAppRoot)) {
+      for (FileStatus appDir : listStatusWithRetries(rmAppRoot)) {
         checkAndResumeUpdateOperation(appDir.getPath());
-        for (FileStatus childNodeStatus : fs.listStatus(appDir.getPath())) {
+        for (FileStatus childNodeStatus :
+            listStatusWithRetries(appDir.getPath())) {
           assert childNodeStatus.isFile();
           String childNodeName = childNodeStatus.getPath().getName();
-          if (checkAndRemovePartialRecord(childNodeStatus.getPath())) {
+          if (checkAndRemovePartialRecordWithRetries(
+              childNodeStatus.getPath())) {
             continue;
           }
-          byte[] childData =
-              readFile(childNodeStatus.getPath(), childNodeStatus.getLen());
+          byte[] childData = readFileWithRetries(childNodeStatus.getPath(),
+              childNodeStatus.getLen());
           if (childNodeName.startsWith(ApplicationId.appIdStrPrefix)) {
             // application
             if (LOG.isDebugEnabled()) {
@@ -292,7 +304,7 @@ public class FileSystemRMStateStore extends RMStateStore {
     // If it does, the prior updateFile is failed on half way. We need to
     // complete replacing the old file first.
     FileStatus[] newChildNodes =
-        fs.listStatus(path, new PathFilter() {
+        listStatusWithRetries(path, new PathFilter() {
       @Override
       public boolean accept(Path path) {
         return path.getName().endsWith(".new");
@@ -310,12 +322,12 @@ public class FileSystemRMStateStore extends RMStateStore {
   }
   private void loadRMDTSecretManagerState(RMState rmState) throws Exception {
     checkAndResumeUpdateOperation(rmDTSecretManagerRoot);
-    FileStatus[] childNodes = fs.listStatus(rmDTSecretManagerRoot);
+    FileStatus[] childNodes = listStatusWithRetries(rmDTSecretManagerRoot);
 
     for(FileStatus childNodeStatus : childNodes) {
       assert childNodeStatus.isFile();
       String childNodeName = childNodeStatus.getPath().getName();
-      if (checkAndRemovePartialRecord(childNodeStatus.getPath())) {
+      if (checkAndRemovePartialRecordWithRetries(childNodeStatus.getPath())) {
         continue;
       }
       if(childNodeName.startsWith(DELEGATION_TOKEN_SEQUENCE_NUMBER_PREFIX)) {
@@ -325,35 +337,36 @@ public class FileSystemRMStateStore extends RMStateStore {
       }
 
       Path childNodePath = getNodePath(rmDTSecretManagerRoot, childNodeName);
-      byte[] childData = readFile(childNodePath, childNodeStatus.getLen());
+      byte[] childData = readFileWithRetries(childNodePath,
+          childNodeStatus.getLen());
       ByteArrayInputStream is = new ByteArrayInputStream(childData);
-      DataInputStream fsIn = new DataInputStream(is);
-      if(childNodeName.startsWith(DELEGATION_KEY_PREFIX)){
-        DelegationKey key = new DelegationKey();
-        key.readFields(fsIn);
-        rmState.rmSecretManagerState.masterKeyState.add(key);
-        if (LOG.isDebugEnabled()) {
-          LOG.debug("Loaded delegation key: keyId=" + key.getKeyId()
-              + ", expirationDate=" + key.getExpiryDate());
-        }
-      } else if (childNodeName.startsWith(DELEGATION_TOKEN_PREFIX)) {
-        RMDelegationTokenIdentifierData identifierData =
-            new RMDelegationTokenIdentifierData();
-        identifierData.readFields(fsIn);
-        RMDelegationTokenIdentifier identifier =
-            identifierData.getTokenIdentifier();
-        long renewDate = identifierData.getRenewDate();
-
-        rmState.rmSecretManagerState.delegationTokenState.put(identifier,
-          renewDate);
-        if (LOG.isDebugEnabled()) {
-          LOG.debug("Loaded RMDelegationTokenIdentifier: " + identifier
-              + " renewDate=" + renewDate);
+      try (DataInputStream fsIn = new DataInputStream(is)) {
+        if (childNodeName.startsWith(DELEGATION_KEY_PREFIX)) {
+          DelegationKey key = new DelegationKey();
+          key.readFields(fsIn);
+          rmState.rmSecretManagerState.masterKeyState.add(key);
+          if (LOG.isDebugEnabled()) {
+            LOG.debug("Loaded delegation key: keyId=" + key.getKeyId()
+                + ", expirationDate=" + key.getExpiryDate());
+          }
+        } else if (childNodeName.startsWith(DELEGATION_TOKEN_PREFIX)) {
+          RMDelegationTokenIdentifierData identifierData =
+              new RMDelegationTokenIdentifierData();
+          identifierData.readFields(fsIn);
+          RMDelegationTokenIdentifier identifier =
+              identifierData.getTokenIdentifier();
+          long renewDate = identifierData.getRenewDate();
+
+          rmState.rmSecretManagerState.delegationTokenState.put(identifier,
+            renewDate);
+          if (LOG.isDebugEnabled()) {
+            LOG.debug("Loaded RMDelegationTokenIdentifier: " + identifier
+                + " renewDate=" + renewDate);
+          }
+        } else {
+          LOG.warn("Unknown file for recovering RMDelegationTokenSecretManager");
         }
-      } else {
-        LOG.warn("Unknown file for recovering RMDelegationTokenSecretManager");
       }
-      fsIn.close();
     }
   }
 
@@ -361,7 +374,7 @@ public class FileSystemRMStateStore extends RMStateStore {
   public synchronized void storeApplicationStateInternal(ApplicationId appId,
       ApplicationStateData appStateDataPB) throws Exception {
     Path appDirPath = getAppDir(rmAppRoot, appId);
-    fs.mkdirs(appDirPath);
+    mkdirsWithRetries(appDirPath);
     Path nodeCreatePath = getNodePath(appDirPath, appId.toString());
 
     LOG.info("Storing info for app: " + appId + " at: " + nodeCreatePath);
@@ -369,7 +382,7 @@ public class FileSystemRMStateStore extends RMStateStore {
     try {
       // currently throw all exceptions. May need to respond differently for HA
       // based on whether we have lost the right to write to FS
-      writeFile(nodeCreatePath, appStateData);
+      writeFileWithRetries(nodeCreatePath, appStateData);
     } catch (Exception e) {
       LOG.info("Error storing info for app: " + appId, e);
       throw e;
@@ -408,7 +421,7 @@ public class FileSystemRMStateStore extends RMStateStore {
     try {
       // currently throw all exceptions. May need to respond differently for HA
       // based on whether we have lost the right to write to FS
-      writeFile(nodeCreatePath, attemptStateData);
+      writeFileWithRetries(nodeCreatePath, attemptStateData);
     } catch (Exception e) {
       LOG.info("Error storing info for attempt: " + appAttemptId, e);
       throw e;
@@ -444,7 +457,7 @@ public class FileSystemRMStateStore extends RMStateStore {
         appState.getApplicationSubmissionContext().getApplicationId();
     Path nodeRemovePath = getAppDir(rmAppRoot, appId);
     LOG.info("Removing info for app: " + appId + " at: " + nodeRemovePath);
-    deleteFile(nodeRemovePath);
+    deleteFileWithRetries(nodeRemovePath);
   }
 
   @Override
@@ -460,7 +473,7 @@ public class FileSystemRMStateStore extends RMStateStore {
     Path nodeCreatePath = getNodePath(rmDTSecretManagerRoot,
       DELEGATION_TOKEN_PREFIX + identifier.getSequenceNumber());
     LOG.info("Removing RMDelegationToken_" + identifier.getSequenceNumber());
-    deleteFile(nodeCreatePath);
+    deleteFileWithRetries(nodeCreatePath);
   }
 
   @Override
@@ -483,7 +496,7 @@ public class FileSystemRMStateStore extends RMStateStore {
       updateFile(nodeCreatePath, identifierData.toByteArray());
     } else {
       LOG.info("Storing RMDelegationToken_" + identifier.getSequenceNumber());
-      writeFile(nodeCreatePath, identifierData.toByteArray());
+      writeFileWithRetries(nodeCreatePath, identifierData.toByteArray());
 
       // store sequence number
       Path latestSequenceNumberPath = getNodePath(rmDTSecretManagerRoot,
@@ -492,11 +505,12 @@ public class FileSystemRMStateStore extends RMStateStore {
       LOG.info("Storing " + DELEGATION_TOKEN_SEQUENCE_NUMBER_PREFIX
           + identifier.getSequenceNumber());
       if (dtSequenceNumberPath == null) {
-        if (!createFile(latestSequenceNumberPath)) {
+        if (!createFileWithRetries(latestSequenceNumberPath)) {
           throw new Exception("Failed to create " + latestSequenceNumberPath);
         }
       } else {
-        if (!renameFile(dtSequenceNumberPath, latestSequenceNumberPath)) {
+        if (!renameFileWithRetries(dtSequenceNumberPath,
+            latestSequenceNumberPath)) {
           throw new Exception("Failed to rename " + dtSequenceNumberPath);
         }
       }
@@ -510,11 +524,11 @@ public class FileSystemRMStateStore extends RMStateStore {
     Path nodeCreatePath = getNodePath(rmDTSecretManagerRoot,
           DELEGATION_KEY_PREFIX + masterKey.getKeyId());
     ByteArrayOutputStream os = new ByteArrayOutputStream();
-    DataOutputStream fsOut = new DataOutputStream(os);
-    LOG.info("Storing RMDelegationKey_" + masterKey.getKeyId());
-    masterKey.write(fsOut);
-    writeFile(nodeCreatePath, os.toByteArray());
-    fsOut.close();
+    try (DataOutputStream fsOut = new DataOutputStream(os)) {
+      LOG.info("Storing RMDelegationKey_" + masterKey.getKeyId());
+      masterKey.write(fsOut);
+      writeFileWithRetries(nodeCreatePath, os.toByteArray());
+    }
   }
 
   @Override
@@ -523,13 +537,13 @@ public class FileSystemRMStateStore extends RMStateStore {
     Path nodeCreatePath = getNodePath(rmDTSecretManagerRoot,
           DELEGATION_KEY_PREFIX + masterKey.getKeyId());
     LOG.info("Removing RMDelegationKey_"+ masterKey.getKeyId());
-    deleteFile(nodeCreatePath);
+    deleteFileWithRetries(nodeCreatePath);
   }
 
   @Override
-  public synchronized void deleteStore() throws IOException {
-    if (fs.exists(rootDirPath)) {
-      fs.delete(rootDirPath, true);
+  public synchronized void deleteStore() throws Exception {
+    if (existsWithRetries(rootDirPath)) {
+      deleteFileWithRetries(rootDirPath);
     }
   }
 
@@ -539,6 +553,146 @@ public class FileSystemRMStateStore extends RMStateStore {
 
   // FileSystem related code
 
+  private boolean checkAndRemovePartialRecordWithRetries(final Path record)
+      throws Exception {
+    return new FSAction<Boolean>() {
+      @Override
+      public Boolean run() throws Exception {
+        return checkAndRemovePartialRecord(record);
+      }
+    }.runWithRetries();
+  }
+
+  private void mkdirsWithRetries(final Path appDirPath) throws Exception {
+    new FSAction<Void>() {
+      @Override
+      public Void run() throws Exception {
+        fs.mkdirs(appDirPath);
+        return null;
+      }
+    }.runWithRetries();
+  }
+
+  private void writeFileWithRetries(final Path outputPath,final byte[] data)
+      throws Exception {
+    new FSAction<Void>() {
+      @Override
+      public Void run() throws Exception {
+        writeFile(outputPath, data);
+        return null;
+      }
+    }.runWithRetries();
+  }
+
+  private void deleteFileWithRetries(final Path deletePath) throws Exception {
+    new FSAction<Void>() {
+      @Override
+      public Void run() throws Exception {
+        deleteFile(deletePath);
+        return null;
+      }
+    }.runWithRetries();
+  }
+
+  private boolean renameFileWithRetries(final Path src, final Path dst)
+      throws Exception {
+    return new FSAction<Boolean>() {
+      @Override
+      public Boolean run() throws Exception {
+        return renameFile(src, dst);
+      }
+    }.runWithRetries();
+  }
+
+  private boolean createFileWithRetries(final Path newFile) throws Exception {
+    return new FSAction<Boolean>() {
+      @Override
+      public Boolean run() throws Exception {
+        return createFile(newFile);
+      }
+    }.runWithRetries();
+  }
+
+  private FileStatus getFileStatusWithRetries(final Path path)
+      throws Exception {
+    return new FSAction<FileStatus>() {
+      @Override
+      public FileStatus run() throws Exception {
+        return getFileStatus(path);
+      }
+    }.runWithRetries();
+  }
+
+  private boolean existsWithRetries(final Path path) throws Exception {
+    return new FSAction<Boolean>() {
+      @Override
+      public Boolean run() throws Exception {
+        return fs.exists(path);
+      }
+    }.runWithRetries();
+  }
+
+  private byte[] readFileWithRetries(final Path inputPath, final long len)
+      throws Exception {
+    return new FSAction<byte[]>() {
+      @Override
+      public byte[] run() throws Exception {
+        return readFile(inputPath, len);
+      }
+    }.runWithRetries();
+  }
+
+  private FileStatus[] listStatusWithRetries(final Path path)
+      throws Exception {
+    return new FSAction<FileStatus[]>() {
+      @Override
+      public FileStatus[] run() throws Exception {
+        return fs.listStatus(path);
+      }
+    }.runWithRetries();
+  }
+
+  private FileStatus[] listStatusWithRetries(final Path path,
+      final PathFilter filter) throws Exception {
+    return new FSAction<FileStatus[]>() {
+      @Override
+      public FileStatus[] run() throws Exception {
+        return fs.listStatus(path, filter);
+      }
+    }.runWithRetries();
+  }
+
+  private void closeWithRetries() throws Exception {
+    new FSAction<Void>() {
+      @Override
+      public Void run() throws Exception {
+        fs.close();
+        return null;
+      }
+    }.runWithRetries();
+  }
+
+  private abstract class FSAction<T> {
+    abstract T run() throws Exception;
+
+    T runWithRetries() throws Exception {
+      int retry = 0;
+      while (true) {
+        try {
+          return run();
+        } catch (IOException e) {
+          LOG.info("Exception while executing a FS operation.", e);
+          if (++retry > fsNumRetries) {
+            LOG.info("Maxed out FS retries. Giving up!");
+            throw e;
+          }
+          LOG.info("Retrying operation on FS. Retry no. " + retry);
+          Thread.sleep(fsRetryInterval);
+        }
+      }
+    }
+  }
+
   private void deleteFile(Path deletePath) throws Exception {
     if(!fs.delete(deletePath, true)) {
       throw new Exception("Failed to delete " + deletePath);
@@ -595,18 +749,18 @@ public class FileSystemRMStateStore extends RMStateStore {
    */
   protected void updateFile(Path outputPath, byte[] data) throws Exception {
     Path newPath = new Path(outputPath.getParent(), outputPath.getName() + ".new");
-    // use writeFile to make sure .new file is created atomically
-    writeFile(newPath, data);
+    // use writeFileWithRetries to make sure .new file is created atomically
+    writeFileWithRetries(newPath, data);
     replaceFile(newPath, outputPath);
   }
 
   protected void replaceFile(Path srcPath, Path dstPath) throws Exception {
-    if (fs.exists(dstPath)) {
-      deleteFile(dstPath);
+    if (existsWithRetries(dstPath)) {
+      deleteFileWithRetries(dstPath);
     } else {
       LOG.info("File doesn't exist. Skip deleting the file " + dstPath);
     }
-    fs.rename(srcPath, dstPath);
+    renameFileWithRetries(srcPath, dstPath);
   }
 
   @Private
@@ -637,8 +791,17 @@ public class FileSystemRMStateStore extends RMStateStore {
     if (isUpdate) {
       updateFile(nodeCreatePath, stateData);
     } else {
-      writeFile(nodeCreatePath, stateData);
+      writeFileWithRetries(nodeCreatePath, stateData);
     }
   }
 
+  @VisibleForTesting
+  public int getNumRetries() {
+    return fsNumRetries;
+  }
+
+  @VisibleForTesting
+  public long getRetryInterval() {
+    return fsRetryInterval;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4e95f987/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/TestFSRMStateStore.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/recovery/TestFSRMStateStore.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/TestFSRMStateStore.java
index d0d19e3..675d73c 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/TestFSRMStateStore.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/TestFSRMStateStore.java
@@ -100,7 +100,12 @@ public class TestFSRMStateStore extends RMStateStoreTestBase {
           workingDirPathURI.toString());
       conf.set(YarnConfiguration.FS_RM_STATE_STORE_RETRY_POLICY_SPEC,
         "100,6000");
+      conf.setInt(YarnConfiguration.FS_RM_STATE_STORE_NUM_RETRIES, 5);
+      conf.setLong(YarnConfiguration.FS_RM_STATE_STORE_RETRY_INTERVAL_MS,
+          900L);
       this.store = new TestFileSystemRMStore(conf);
+      Assert.assertEquals(store.getNumRetries(), 5);
+      Assert.assertEquals(store.getRetryInterval(), 900L);
       return store;
     }
 


[30/50] [abbrv] hadoop git commit: YARN-3168. Convert site documentation from apt to markdown (Gururaj Shetty via aw)

Posted by zh...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/06aca7c6/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/ResourceManagerRestart.md
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/ResourceManagerRestart.md b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/ResourceManagerRestart.md
new file mode 100644
index 0000000..e516afb
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/ResourceManagerRestart.md
@@ -0,0 +1,181 @@
+<!---
+  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.
+-->
+
+ResourceManger Restart
+======================
+
+* [Overview](#Overview)
+* [Feature](#Feature)
+* [Configurations](#Configurations)
+    * [Enable RM Restart](#Enable_RM_Restart)
+    * [Configure the state-store for persisting the RM state](#Configure_the_state-store_for_persisting_the_RM_state)
+    * [How to choose the state-store implementation](#How_to_choose_the_state-store_implementation)
+    * [Configurations for Hadoop FileSystem based state-store implementation](#Configurations_for_Hadoop_FileSystem_based_state-store_implementation)
+    * [Configurations for ZooKeeper based state-store implementation](#Configurations_for_ZooKeeper_based_state-store_implementation)
+    * [Configurations for LevelDB based state-store implementation](#Configurations_for_LevelDB_based_state-store_implementation)
+    * [Configurations for work-preserving RM recovery](#Configurations_for_work-preserving_RM_recovery)
+* [Notes](#Notes)
+* [Sample Configurations](#Sample_Configurations)
+
+Overview
+--------
+
+ResourceManager is the central authority that manages resources and schedules applications running atop of YARN. Hence, it is potentially a single point of failure in a Apache YARN cluster.
+`
+This document gives an overview of ResourceManager Restart, a feature that enhances ResourceManager to keep functioning across restarts and also makes ResourceManager down-time invisible to end-users.
+
+ResourceManager Restart feature is divided into two phases: 
+
+* **ResourceManager Restart Phase 1 (Non-work-preserving RM restart)**: Enhance RM to persist application/attempt state and other credentials information in a pluggable state-store. RM will reload this information from state-store upon restart and re-kick the previously running applications. Users are not required to re-submit the applications.
+
+* **ResourceManager Restart Phase 2 (Work-preserving RM restart)**: Focus on re-constructing the running state of ResourceManager by combining the container statuses from NodeManagers and container requests from ApplicationMasters upon restart. The key difference from phase 1 is that previously running applications will not be killed after RM restarts, and so applications won't lose its work because of RM outage.
+
+Feature
+-------
+
+* **Phase 1: Non-work-preserving RM restart** 
+
+     As of Hadoop 2.4.0 release, only ResourceManager Restart Phase 1 is implemented which is described below.
+
+     The overall concept is that RM will persist the application metadata (i.e. ApplicationSubmissionContext) in a pluggable state-store when client submits an application and also saves the final status of the application such as the completion state (failed, killed, finished) and diagnostics when the application completes. Besides, RM also saves the credentials like security keys, tokens to work in a secure  environment. Any time RM shuts down, as long as the required information (i.e.application metadata and the alongside credentials if running in a secure environment) is available in the state-store, when RM restarts, it can pick up the application metadata from the state-store and re-submit the application. RM won't re-submit the applications if they were already completed (i.e. failed, killed, finished) before RM went down.
+
+     NodeManagers and clients during the down-time of RM will keep polling RM until RM comes up. When RM becomes alive, it will send a re-sync command to all the NodeManagers and ApplicationMasters it was talking to via heartbeats. As of Hadoop 2.4.0 release, the behaviors for NodeManagers and ApplicationMasters to handle this command are: NMs will kill all its managed containers and re-register with RM. From the RM's perspective, these re-registered NodeManagers are similar to the newly joining NMs. AMs(e.g. MapReduce AM) are expected to shutdown when they receive the re-sync command. After RM restarts and loads all the application metadata, credentials from state-store and populates them into memory, it will create a new attempt (i.e. ApplicationMaster) for each application that was not yet completed and re-kick that application as usual. As described before, the previously running applications' work is lost in this manner since they are essentially killed by RM via the re-sync co
 mmand on restart.
+
+* **Phase 2: Work-preserving RM restart** 
+
+     As of Hadoop 2.6.0, we further enhanced RM restart feature to address the problem to not kill any applications running on YARN cluster if RM restarts.
+
+     Beyond all the groundwork that has been done in Phase 1 to ensure the persistency of application state and reload that state on recovery, Phase 2 primarily focuses on re-constructing the entire running state of YARN cluster, the majority of which is the state of the central scheduler inside RM which keeps track of all containers' life-cycle, applications' headroom and resource requests, queues' resource usage etc. In this way, RM doesn't need to kill the AM and re-run the application from scratch as it is done in Phase 1. Applications can simply re-sync back with RM and resume from where it were left off.
+
+     RM recovers its runing state by taking advantage of the container statuses sent from all NMs. NM will not kill the containers when it re-syncs with the restarted RM. It continues managing the containers and send the container statuses across to RM when it re-registers. RM reconstructs the container instances and the associated applications' scheduling status by absorbing these containers' information. In the meantime, AM needs to re-send the outstanding resource requests to RM because RM may lose the unfulfilled requests when it shuts down. Application writers using AMRMClient library to communicate with RM do not need to worry about the part of AM re-sending resource requests to RM on re-sync, as it is automatically taken care by the library itself.
+
+Configurations
+--------------
+
+This section describes the configurations involved to enable RM Restart feature.
+
+### Enable RM Restart
+
+| Property | Description |
+|:---- |:---- |
+| `yarn.resourcemanager.recovery.enabled` | `true` |
+
+### Configure the state-store for persisting the RM state
+
+| Property | Description |
+|:---- |:---- |
+| `yarn.resourcemanager.store.class` | The class name of the state-store to be used for saving application/attempt state and the credentials. The available state-store implementations are `org.apache.hadoop.yarn.server.resourcemanager.recovery.ZKRMStateStore`, a ZooKeeper based state-store implementation and `org.apache.hadoop.yarn.server.resourcemanager.recovery.FileSystemRMStateStore`, a Hadoop FileSystem based state-store implementation like HDFS and local FS. `org.apache.hadoop.yarn.server.resourcemanager.recovery.LeveldbRMStateStore`, a LevelDB based state-store implementation. The default value is set to `org.apache.hadoop.yarn.server.resourcemanager.recovery.FileSystemRMStateStore`. |
+
+### How to choose the state-store implementation
+
+   * **ZooKeeper based state-store**: User is free to pick up any storage to set up RM restart, but must use ZooKeeper based state-store to support RM HA. The reason is that only ZooKeeper based state-store supports fencing mechanism to avoid a split-brain situation where multiple RMs assume they are active and can edit the state-store at the same time.
+
+   * **FileSystem based state-store**: HDFS and local FS based state-store are supported. Fencing mechanism is not supported.
+
+   * **LevelDB based state-store**: LevelDB based state-store is considered more light weight than HDFS and ZooKeeper based state-store. LevelDB supports better atomic operations, fewer I/O ops per state update,
+    and far fewer total files on the filesystem. Fencing mechanism is not supported.
+
+### Configurations for Hadoop FileSystem based state-store implementation
+
+   Support both HDFS and local FS based state-store implementation. The type of file system to be used is determined by the scheme of URI. e.g. `hdfs://localhost:9000/rmstore` uses HDFS as the storage and `file:///tmp/yarn/rmstore` uses local FS as the storage. If no scheme(`hdfs://` or `file://`) is specified in the URI, the type of storage to be used is determined by `fs.defaultFS` defined in `core-site.xml`.
+
+* Configure the URI where the RM state will be saved in the Hadoop FileSystem state-store.
+
+| Property | Description |
+|:---- |:---- |
+| `yarn.resourcemanager.fs.state-store.uri` | URI pointing to the location of the FileSystem path where RM state will be stored (e.g. hdfs://localhost:9000/rmstore). Default value is `${hadoop.tmp.dir}/yarn/system/rmstore`. If FileSystem name is not provided, `fs.default.name` specified in **conf/core-site.xml* will be used. |
+
+* Configure the retry policy state-store client uses to connect with the Hadoop FileSystem.
+
+| Property | Description |
+|:---- |:---- |
+| `yarn.resourcemanager.fs.state-store.retry-policy-spec` | Hadoop FileSystem client retry policy specification. Hadoop FileSystem client retry is always enabled. Specified in pairs of sleep-time and number-of-retries i.e. (t0, n0), (t1, n1), ..., the first n0 retries sleep t0 milliseconds on average, the following n1 retries sleep t1 milliseconds on average, and so on. Default value is (2000, 500) |
+
+### Configurations for ZooKeeper based state-store implementation
+  
+* Configure the ZooKeeper server address and the root path where the RM state is stored.
+
+| Property | Description |
+|:---- |:---- |
+| `yarn.resourcemanager.zk-address` | Comma separated list of Host:Port pairs. Each corresponds to a ZooKeeper server (e.g. "127.0.0.1:3000,127.0.0.1:3001,127.0.0.1:3002") to be used by the RM for storing RM state. |
+| `yarn.resourcemanager.zk-state-store.parent-path` | The full path of the root znode where RM state will be stored. Default value is /rmstore. |
+
+* Configure the retry policy state-store client uses to connect with the ZooKeeper server.
+
+| Property | Description |
+|:---- |:---- |
+| `yarn.resourcemanager.zk-num-retries` | Number of times RM tries to connect to ZooKeeper server if the connection is lost. Default value is 500. |
+| `yarn.resourcemanager.zk-retry-interval-ms` | The interval in milliseconds between retries when connecting to a ZooKeeper server. Default value is 2 seconds. |
+| `yarn.resourcemanager.zk-timeout-ms` | ZooKeeper session timeout in milliseconds. This configuration is used by the ZooKeeper server to determine when the session expires. Session expiration happens when the server does not hear from the client (i.e. no heartbeat) within the session timeout period specified by this configuration. Default value is 10 seconds |
+
+* Configure the ACLs to be used for setting permissions on ZooKeeper znodes.
+
+| Property | Description |
+|:---- |:---- |
+| `yarn.resourcemanager.zk-acl` | ACLs to be used for setting permissions on ZooKeeper znodes. Default value is `world:anyone:rwcda` |
+
+### Configurations for LevelDB based state-store implementation
+
+| Property | Description |
+|:---- |:---- |
+| `yarn.resourcemanager.leveldb-state-store.path` | Local path where the RM state will be stored. Default value is `${hadoop.tmp.dir}/yarn/system/rmstore` |
+
+### Configurations for work-preserving RM recovery
+
+| Property | Description |
+|:---- |:---- |
+| `yarn.resourcemanager.work-preserving-recovery.scheduling-wait-ms` | Set the amount of time RM waits before allocating new containers on RM work-preserving recovery. Such wait period gives RM a chance to settle down resyncing with NMs in the cluster on recovery, before assigning new containers to applications.|
+
+Notes
+-----
+
+ContainerId string format is changed if RM restarts with work-preserving recovery enabled. It used to be such format:
+
+    Container_{clusterTimestamp}_{appId}_{attemptId}_{containerId}, e.g. Container_1410901177871_0001_01_000005.
+
+It is now changed to:
+
+    Container_e{epoch}_{clusterTimestamp}_{appId}_{attemptId}_{containerId}, e.g. Container_e17_1410901177871_0001_01_000005.
+ 
+Here, the additional epoch number is a monotonically increasing integer which starts from 0 and is increased by 1 each time RM restarts. If epoch number is 0, it is omitted and the containerId string format stays the same as before.
+
+Sample Configurations
+---------------------
+
+Below is a minimum set of configurations for enabling RM work-preserving restart using ZooKeeper based state store.
+
+
+     <property>
+       <description>Enable RM to recover state after starting. If true, then 
+       yarn.resourcemanager.store.class must be specified</description>
+       <name>yarn.resourcemanager.recovery.enabled</name>
+       <value>true</value>
+     </property>
+   
+     <property>
+       <description>The class to use as the persistent store.</description>
+       <name>yarn.resourcemanager.store.class</name>
+       <value>org.apache.hadoop.yarn.server.resourcemanager.recovery.ZKRMStateStore</value>
+     </property>
+
+     <property>
+       <description>Comma separated list of Host:Port pairs. Each corresponds to a ZooKeeper server
+       (e.g. "127.0.0.1:3000,127.0.0.1:3001,127.0.0.1:3002") to be used by the RM for storing RM state.
+       This must be supplied when using org.apache.hadoop.yarn.server.resourcemanager.recovery.ZKRMStateStore
+       as the value for yarn.resourcemanager.store.class</description>
+       <name>yarn.resourcemanager.zk-address</name>
+       <value>127.0.0.1:2181</value>
+     </property>
+
+

http://git-wip-us.apache.org/repos/asf/hadoop/blob/06aca7c6/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/SecureContainer.md
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/SecureContainer.md b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/SecureContainer.md
new file mode 100644
index 0000000..f32e460
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/SecureContainer.md
@@ -0,0 +1,135 @@
+<!---
+  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.
+-->
+
+YARN Secure Containers
+======================
+
+* [Overview](#Overview)
+
+Overview
+--------
+
+YARN containers in a secure cluster use the operating system facilities to offer execution isolation for containers. Secure containers execute under the credentials of the job user. The operating system enforces access restriction for the container. The container must run as the use that submitted the application.
+
+Secure Containers work only in the context of secured YARN clusters.
+
+###Container isolation requirements
+
+  The container executor must access the local files and directories needed by the container such as jars, configuration files, log files, shared objects etc. Although it is launched by the NodeManager, the container should not have access to the NodeManager private files and configuration. Container running applications submitted by different users should be isolated and unable to access each other files and directories. Similar requirements apply to other system non-file securable objects like named pipes, critical sections, LPC queues, shared memory etc.
+
+###Linux Secure Container Executor
+
+  On Linux environment the secure container executor is the `LinuxContainerExecutor`. It uses an external program called the **container-executor**\> to launch the container. This program has the `setuid` access right flag set which allows it to launch the container with the permissions of the YARN application user.
+
+###Configuration
+
+  The configured directories for `yarn.nodemanager.local-dirs` and `yarn.nodemanager.log-dirs` must be owned by the configured NodeManager user (`yarn`) and group (`hadoop`). The permission set on these directories must be `drwxr-xr-x`.
+
+  The `container-executor` program must be owned by `root` and have the permission set `---sr-s---`.
+
+  To configure the `NodeManager` to use the `LinuxContainerExecutor` set the following in the **conf/yarn-site.xml**:
+
+```xml
+<property>
+  <name>yarn.nodemanager.container-executor.class</name>
+  <value>org.apache.hadoop.yarn.server.nodemanager.LinuxContainerExecutor</value>
+</property>
+
+<property>
+  <name>yarn.nodemanager.linux-container-executor.group</name>
+  <value>hadoop</value>
+</property>
+```
+
+  Additionally the LCE requires the `container-executor.cfg` file, which is read by the `container-executor` program.
+
+```
+yarn.nodemanager.linux-container-executor.group=#configured value of yarn.nodemanager.linux-container-executor.group
+banned.users=#comma separated list of users who can not run applications
+allowed.system.users=#comma separated list of allowed system users
+min.user.id=1000#Prevent other super-users
+```
+
+###Windows Secure Container Executor (WSCE)
+
+  The Windows environment secure container executor is the `WindowsSecureContainerExecutor`. It uses the Windows S4U infrastructure to launch the container as the YARN application user. The WSCE requires the presense of the `hadoopwinutilsvc` service. This services is hosted by `%HADOOP_HOME%\bin\winutils.exe` started with the `service` command line argument. This service offers some privileged operations that require LocalSystem authority so that the NM is not required to run the entire JVM and all the NM code in an elevated context. The NM interacts with the `hadoopwintulsvc` service by means of Local RPC (LRPC) via calls JNI to the RCP client hosted in `hadoop.dll`.
+
+###Configuration
+
+  To configure the `NodeManager` to use the `WindowsSecureContainerExecutor` set the following in the **conf/yarn-site.xml**:
+
+```xml
+        <property>
+          <name>yarn.nodemanager.container-executor.class</name>
+          <value>org.apache.hadoop.yarn.server.nodemanager.WindowsSecureContainerExecutor</value>
+        </property>
+
+        <property>
+          <name>yarn.nodemanager.windows-secure-container-executor.group</name>
+          <value>yarn</value>
+        </property>
+```
+   
+  The hadoopwinutilsvc uses `%HADOOP_HOME%\etc\hadoop\wsce_site.xml` to configure access to the privileged operations.
+
+```xml
+<property>
+ <name>yarn.nodemanager.windows-secure-container-executor.impersonate.allowed</name>
+  <value>HadoopUsers</value>
+</property>
+
+<property>
+  <name>yarn.nodemanager.windows-secure-container-executor.impersonate.denied</name>
+  <value>HadoopServices,Administrators</value>
+</property>
+
+<property>
+  <name>yarn.nodemanager.windows-secure-container-executor.allowed</name>
+  <value>nodemanager</value>
+</property>
+
+<property>
+  <name>yarn.nodemanager.windows-secure-container-executor.local-dirs</name>
+  <value>nm-local-dir, nm-log-dirs</value>
+</property>
+
+<property>
+  <name>yarn.nodemanager.windows-secure-container-executor.job-name</name>
+  <value>nodemanager-job-name</value>
+</property>  
+```
+
+  `yarn.nodemanager.windows-secure-container-executor.allowed` should contain the name of the service account running the nodemanager. This user will be allowed to access the hadoopwintuilsvc functions.
+
+  `yarn.nodemanager.windows-secure-container-executor.impersonate.allowed` should contain the users that are allowed to create containers in the cluster. These users will be allowed to be impersonated by hadoopwinutilsvc.
+
+  `yarn.nodemanager.windows-secure-container-executor.impersonate.denied` should contain users that are explictly forbiden from creating containers. hadoopwinutilsvc will refuse to impersonate these users.
+
+  `yarn.nodemanager.windows-secure-container-executor.local-dirs` should contain the nodemanager local dirs. hadoopwinutilsvc will allow only file operations under these directories. This should contain the same values as `$yarn.nodemanager.local-dirs, $yarn.nodemanager.log-dirs` but note that hadoopwinutilsvc XML configuration processing does not do substitutions so the value must be the final value. All paths must be absolute and no environment variable substitution will be performed. The paths are compared LOCAL\_INVARIANT case insensitive string comparison, the file path validated must start with one of the paths listed in local-dirs configuration. Use comma as path separator:`,`
+
+  `yarn.nodemanager.windows-secure-container-executor.job-name` should contain an Windows NT job name that all containers should be added to. This configuration is optional. If not set, the container is not added to a global NodeManager job. Normally this should be set to the job that the NM is assigned to, so that killing the NM kills also all containers. Hadoopwinutilsvc will not attempt to create this job, the job must exists when the container is launched. If the value is set and the job does not exists, container launch will fail with error 2 `The system cannot find the file specified`. Note that this global NM job is not related to the container job, which always gets created for each container and is named after the container ID. This setting controls a global job that spans all containers and the parent NM, and as such it requires nested jobs. Nested jobs are available only post Windows 8 and Windows Server 2012.
+
+####Useful Links
+
+  * [Exploring S4U Kerberos Extensions in Windows Server 2003](http://msdn.microsoft.com/en-us/magazine/cc188757.aspx)
+
+  * [Nested Jobs](http://msdn.microsoft.com/en-us/library/windows/desktop/hh448388.aspx)
+
+  * [Winutils needs ability to create task as domain user](https://issues.apache.org/jira/browse/YARN-1063)
+
+  * [Implement secure Windows Container Executor](https://issues.apache.org/jira/browse/YARN-1972)
+
+  * [Remove the need to run NodeManager as privileged account for Windows Secure Container Executor](https://issues.apache.org/jira/browse/YARN-2198)
+
+

http://git-wip-us.apache.org/repos/asf/hadoop/blob/06aca7c6/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/TimelineServer.md
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/TimelineServer.md b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/TimelineServer.md
new file mode 100644
index 0000000..4889936
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/TimelineServer.md
@@ -0,0 +1,231 @@
+<!---
+  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.
+-->
+
+YARN Timeline Server
+====================
+
+* [Overview](#Overview)
+* [Current Status](#Current_Status)
+* [Basic Configuration](#Basic_Configuration)
+* [Advanced Configuration](#Advanced_Configuration)
+* [Generic-data related Configuration](#Generic-data_related_Configuration)
+* [Per-framework-date related Configuration](#Per-framework-date_related_Configuration)
+* [Running Timeline server](#Running_Timeline_server)
+* [Accessing generic-data via command-line](#Accessing_generic-data_via_command-line)
+* [Publishing of per-framework data by applications](#Publishing_of_per-framework_data_by_applications)
+
+Overview
+--------
+
+Storage and retrieval of applications' current as well as historic information in a generic fashion is solved in YARN through the Timeline Server (previously also called Generic Application History Server). This serves two responsibilities:
+
+* Generic information about completed applications
+    
+    Generic information includes application level data like queue-name, user information etc in the ApplicationSubmissionContext, list of application-attempts that ran for an application, information about each application-attempt, list of containers run under each application-attempt, and information about each container. Generic data is stored by ResourceManager to a history-store (default implementation on a file-system) and used by the web-UI to display information about completed applications.
+
+* Per-framework information of running and completed applications
+    
+    Per-framework information is completely specific to an application or framework. For example, Hadoop MapReduce framework can include pieces of information like number of map tasks, reduce tasks, counters etc. Application developers can publish the specific information to the Timeline server via TimelineClient from within a client, the ApplicationMaster and/or the application's containers. This information is then queryable via REST APIs for rendering by application/framework specific UIs.
+
+Current Status
+--------------
+
+Timeline sever is a work in progress. The basic storage and retrieval of information, both generic and framework specific, are in place. Timeline server doesn't work in secure mode yet. The generic information and the per-framework information are today collected and presented separately and thus are not integrated well together. Finally, the per-framework information is only available via RESTful APIs, using JSON type content - ability to install framework specific UIs in YARN isn't supported yet.
+
+Basic Configuration
+-------------------
+
+Users need to configure the Timeline server before starting it. The simplest configuration you should add in `yarn-site.xml` is to set the hostname of the Timeline server.
+
+```xml
+<property>
+  <description>The hostname of the Timeline service web application.</description>
+  <name>yarn.timeline-service.hostname</name>
+  <value>0.0.0.0</value>
+</property>
+```
+
+Advanced Configuration
+----------------------
+
+In addition to the hostname, admins can also configure whether the service is enabled or not, the ports of the RPC and the web interfaces, and the number of RPC handler threads.
+
+```xml
+<property>
+  <description>Address for the Timeline server to start the RPC server.</description>
+  <name>yarn.timeline-service.address</name>
+  <value>${yarn.timeline-service.hostname}:10200</value>
+</property>
+
+<property>
+  <description>The http address of the Timeline service web application.</description>
+  <name>yarn.timeline-service.webapp.address</name>
+  <value>${yarn.timeline-service.hostname}:8188</value>
+</property>
+
+<property>
+  <description>The https address of the Timeline service web application.</description>
+  <name>yarn.timeline-service.webapp.https.address</name>
+  <value>${yarn.timeline-service.hostname}:8190</value>
+</property>
+
+<property>
+  <description>Handler thread count to serve the client RPC requests.</description>
+  <name>yarn.timeline-service.handler-thread-count</name>
+  <value>10</value>
+</property>
+
+<property>
+  <description>Enables cross-origin support (CORS) for web services where
+  cross-origin web response headers are needed. For example, javascript making
+  a web services request to the timeline server.</description>
+  <name>yarn.timeline-service.http-cross-origin.enabled</name>
+  <value>false</value>
+</property>
+
+<property>
+  <description>Comma separated list of origins that are allowed for web
+  services needing cross-origin (CORS) support. Wildcards (*) and patterns
+  allowed</description>
+  <name>yarn.timeline-service.http-cross-origin.allowed-origins</name>
+  <value>*</value>
+</property>
+
+<property>
+  <description>Comma separated list of methods that are allowed for web
+  services needing cross-origin (CORS) support.</description>
+  <name>yarn.timeline-service.http-cross-origin.allowed-methods</name>
+  <value>GET,POST,HEAD</value>
+</property>
+
+<property>
+  <description>Comma separated list of headers that are allowed for web
+  services needing cross-origin (CORS) support.</description>
+  <name>yarn.timeline-service.http-cross-origin.allowed-headers</name>
+  <value>X-Requested-With,Content-Type,Accept,Origin</value>
+</property>
+
+<property>
+  <description>The number of seconds a pre-flighted request can be cached
+  for web services needing cross-origin (CORS) support.</description>
+  <name>yarn.timeline-service.http-cross-origin.max-age</name>
+  <value>1800</value>
+</property>
+```
+
+Generic-data related Configuration
+----------------------------------
+
+Users can specify whether the generic data collection is enabled or not, and also choose the storage-implementation class for the generic data. There are more configurations related to generic data collection, and users can refer to `yarn-default.xml` for all of them.
+
+```xml
+<property>
+  <description>Indicate to ResourceManager as well as clients whether
+  history-service is enabled or not. If enabled, ResourceManager starts
+  recording historical data that Timelien service can consume. Similarly,
+  clients can redirect to the history service when applications
+  finish if this is enabled.</description>
+  <name>yarn.timeline-service.generic-application-history.enabled</name>
+  <value>false</value>
+</property>
+
+<property>
+  <description>Store class name for history store, defaulting to file system
+  store</description>
+  <name>yarn.timeline-service.generic-application-history.store-class</name>
+  <value>org.apache.hadoop.yarn.server.applicationhistoryservice.FileSystemApplicationHistoryStore</value>
+</property>
+```
+
+Per-framework-date related Configuration
+----------------------------------------
+
+Users can specify whether per-framework data service is enabled or not, choose the store implementation for the per-framework data, and tune the retention of the per-framework data. There are more configurations related to per-framework data service, and users can refer to `yarn-default.xml` for all of them.
+
+```xml
+<property>
+  <description>Indicate to clients whether Timeline service is enabled or not.
+  If enabled, the TimelineClient library used by end-users will post entities
+  and events to the Timeline server.</description>
+  <name>yarn.timeline-service.enabled</name>
+  <value>true</value>
+</property>
+
+<property>
+  <description>Store class name for timeline store.</description>
+  <name>yarn.timeline-service.store-class</name>
+  <value>org.apache.hadoop.yarn.server.timeline.LeveldbTimelineStore</value>
+</property>
+
+<property>
+  <description>Enable age off of timeline store data.</description>
+  <name>yarn.timeline-service.ttl-enable</name>
+  <value>true</value>
+</property>
+
+<property>
+  <description>Time to live for timeline store data in milliseconds.</description>
+  <name>yarn.timeline-service.ttl-ms</name>
+  <value>604800000</value>
+</property>
+```
+
+Running Timeline server
+-----------------------
+
+Assuming all the aforementioned configurations are set properly, admins can start the Timeline server/history service with the following command:
+
+      $ yarn timelineserver
+
+Or users can start the Timeline server / history service as a daemon:
+
+      $ yarn --daemon start timelineserver
+
+Accessing generic-data via command-line
+---------------------------------------
+
+Users can access applications' generic historic data via the command line as below. Note that the same commands are usable to obtain the corresponding information about running applications.
+
+```
+      $ yarn application -status <Application ID>
+      $ yarn applicationattempt -list <Application ID>
+      $ yarn applicationattempt -status <Application Attempt ID>
+      $ yarn container -list <Application Attempt ID>
+      $ yarn container -status <Container ID>
+```
+
+Publishing of per-framework data by applications
+------------------------------------------------
+
+Developers can define what information they want to record for their applications by composing `TimelineEntity` and `TimelineEvent` objects, and put the entities and events to the Timeline server via `TimelineClient`. Following is an example:
+
+```java
+// Create and start the Timeline client
+TimelineClient client = TimelineClient.createTimelineClient();
+client.init(conf);
+client.start();
+
+TimelineEntity entity = null;
+// Compose the entity
+try {
+  TimelinePutResponse response = client.putEntities(entity);
+} catch (IOException e) {
+  // Handle the exception
+} catch (YarnException e) {
+  // Handle the exception
+}
+
+// Stop the Timeline client
+client.stop();
+```

http://git-wip-us.apache.org/repos/asf/hadoop/blob/06aca7c6/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/WebApplicationProxy.md
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/WebApplicationProxy.md b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/WebApplicationProxy.md
new file mode 100644
index 0000000..8d6187d
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/WebApplicationProxy.md
@@ -0,0 +1,24 @@
+<!---
+  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.
+-->
+
+Web Application Proxy
+=====================
+
+The Web Application Proxy is part of YARN. By default it will run as part of the Resource Manager(RM), but can be configured to run in stand alone mode. The reason for the proxy is to reduce the possibility of web based attacks through YARN.
+
+In YARN the Application Master(AM) has the responsibility to provide a web UI and to send that link to the RM. This opens up a number of potential issues. The RM runs as a trusted user, and people visiting that web address will treat it, and links it provides to them as trusted, when in reality the AM is running as a non-trusted user, and the links it gives to the RM could point to anything malicious or otherwise. The Web Application Proxy mitigates this risk by warning users that do not own the given application that they are connecting to an untrusted site.
+
+In addition to this the proxy also tries to reduce the impact that a malicious AM could have on a user. It primarily does this by stripping out cookies from the user, and replacing them with a single cookie providing the user name of the logged in user. This is because most web based authentication systems will identify a user based off of a cookie. By providing this cookie to an untrusted application it opens up the potential for an exploit. If the cookie is designed properly that potential should be fairly minimal, but this is just to reduce that potential attack vector. The current proxy implementation does nothing to prevent the AM from providing links to malicious external sites, nor does it do anything to prevent malicious javascript code from running as well. In fact javascript can be used to get the cookies, so stripping the cookies from the request has minimal benefit at this time.
+
+In the future we hope to address the attack vectors described above and make attaching to an AM's web UI safer.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/06aca7c6/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/WebServicesIntro.md
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/WebServicesIntro.md b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/WebServicesIntro.md
new file mode 100644
index 0000000..0e89a50
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/WebServicesIntro.md
@@ -0,0 +1,569 @@
+<!---
+  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 YARN - Introduction to the web services REST API's
+==========================================================
+
+* [Overview](#Overview)
+* [URI's](#URIs)
+* [HTTP Requests](#HTTP_Requests)
+    * [Summary of HTTP operations](#Summary_of_HTTP_operations)
+    * [Security](#Security)
+    * [Headers Supported](#Headers_Supported)
+* [HTTP Responses](#HTTP_Responses)
+    * [Compression](#Compression)
+    * [Response Formats](#Response_Formats)
+    * [Response Errors](#Response_Errors)
+    * [Response Examples](#Response_Examples)
+* [Sample Usage](#Sample_Usage)
+
+Overview
+--------
+
+The Hadoop YARN web service REST APIs are a set of URI resources that give access to the cluster, nodes, applications, and application historical information. The URI resources are grouped into APIs based on the type of information returned. Some URI resources return collections while others return singletons.
+
+URI's
+-----
+
+The URIs for the REST-based Web services have the following syntax:
+
+      http://{http address of service}/ws/{version}/{resourcepath}
+
+The elements in this syntax are as follows:
+
+      {http address of service} - The http address of the service to get information about. 
+                                  Currently supported are the ResourceManager, NodeManager, 
+                                  MapReduce application master, and history server.
+      {version} - The version of the APIs. In this release, the version is v1.
+      {resourcepath} - A path that defines a singleton resource or a collection of resources. 
+
+HTTP Requests
+-------------
+
+To invoke a REST API, your application calls an HTTP operation on the URI associated with a resource.
+
+### Summary of HTTP operations
+
+Currently only GET is supported. It retrieves information about the resource specified.
+
+### Security
+
+The web service REST API's go through the same security as the web UI. If your cluster adminstrators have filters enabled you must authenticate via the mechanism they specified.
+
+### Headers Supported
+
+      * Accept 
+      * Accept-Encoding
+
+Currently the only fields used in the header is `Accept` and `Accept-Encoding`. `Accept` currently supports XML and JSON for the response type you accept. `Accept-Encoding` currently supports only gzip format and will return gzip compressed output if this is specified, otherwise output is uncompressed. All other header fields are ignored.
+
+HTTP Responses
+--------------
+
+The next few sections describe some of the syntax and other details of the HTTP Responses of the web service REST APIs.
+
+### Compression
+
+This release supports gzip compression if you specify gzip in the Accept-Encoding header of the HTTP request (Accept-Encoding: gzip).
+
+### Response Formats
+
+This release of the web service REST APIs supports responses in JSON and XML formats. JSON is the default. To set the response format, you can specify the format in the Accept header of the HTTP request.
+
+As specified in HTTP Response Codes, the response body can contain the data that represents the resource or an error message. In the case of success, the response body is in the selected format, either JSON or XML. In the case of error, the resonse body is in either JSON or XML based on the format requested. The Content-Type header of the response contains the format requested. If the application requests an unsupported format, the response status code is 500. Note that the order of the fields within response body is not specified and might change. Also, additional fields might be added to a response body. Therefore, your applications should use parsing routines that can extract data from a response body in any order.
+
+### Response Errors
+
+After calling an HTTP request, an application should check the response status code to verify success or detect an error. If the response status code indicates an error, the response body contains an error message. The first field is the exception type, currently only RemoteException is returned. The following table lists the items within the RemoteException error message:
+
+|      Item | Data Type |          Description |
+|:---- |:---- |:---- |
+|   exception |   String |         Exception type |
+| javaClassName |   String |  Java class name of exception |
+|    message |   String | Detailed message of exception |
+
+### Response Examples
+
+#### JSON response with single resource
+
+HTTP Request: GET http://rmhost.domain:8088/ws/v1/cluster/app/application\_1324057493980\_0001
+
+Response Status Line: HTTP/1.1 200 OK
+
+Response Header:
+
+      HTTP/1.1 200 OK
+      Content-Type: application/json
+      Transfer-Encoding: chunked
+      Server: Jetty(6.1.26)
+
+Response Body:
+
+```json
+{
+  app":
+  {
+    "id":"application_1324057493980_0001",
+    "user":"user1",
+    "name":"",
+    "queue":"default",
+    "state":"ACCEPTED",
+    "finalStatus":"UNDEFINED",
+    "progress":0,
+    "trackingUI":"UNASSIGNED",
+    "diagnostics":"",
+    "clusterId":1324057493980,
+    "startedTime":1324057495921,
+    "finishedTime":0,
+    "elapsedTime":2063,
+    "amContainerLogs":"http:\/\/amNM:2\/node\/containerlogs\/container_1324057493980_0001_01_000001",
+    "amHostHttpAddress":"amNM:2"
+  }
+}
+```
+
+#### JSON response with Error response
+
+Here we request information about an application that doesn't exist yet.
+
+HTTP Request: GET http://rmhost.domain:8088/ws/v1/cluster/app/application\_1324057493980\_9999
+
+Response Status Line: HTTP/1.1 404 Not Found
+
+Response Header:
+
+      HTTP/1.1 404 Not Found
+      Content-Type: application/json
+      Transfer-Encoding: chunked
+      Server: Jetty(6.1.26)
+
+Response Body:
+
+```json
+{
+   "RemoteException" : {
+      "javaClassName" : "org.apache.hadoop.yarn.webapp.NotFoundException",
+      "exception" : "NotFoundException",
+      "message" : "java.lang.Exception: app with id: application_1324057493980_9999 not found"
+   }
+}
+```
+
+Sample Usage
+-------------
+
+You can use any number of ways/languages to use the web services REST API's. This example uses the curl command line interface to do the REST GET calls.
+
+In this example, a user submits a MapReduce application to the ResourceManager using a command like:
+
+      hadoop jar hadoop-mapreduce-test.jar sleep -Dmapred.job.queue.name=a1 -m 1 -r 1 -rt 1200000 -mt 20
+
+The client prints information about the job submitted along with the application id, similar to:
+
+    12/01/18 04:25:15 INFO mapred.ResourceMgrDelegate: Submitted application application_1326821518301_0010 to ResourceManager at host.domain.com/10.10.10.10:8032
+    12/01/18 04:25:15 INFO mapreduce.Job: Running job: job_1326821518301_0010
+    12/01/18 04:25:21 INFO mapred.ClientServiceDelegate: The url to track the job: host.domain.com:8088/proxy/application_1326821518301_0010/
+    12/01/18 04:25:22 INFO mapreduce.Job: Job job_1326821518301_0010 running in uber mode : false
+    12/01/18 04:25:22 INFO mapreduce.Job:  map 0% reduce 0%
+
+The user then wishes to track the application. The users starts by getting the information about the application from the ResourceManager. Use the --comopressed option to request output compressed. curl handles uncompressing on client side.
+
+    curl --compressed -H "Accept: application/json" -X GET "http://host.domain.com:8088/ws/v1/cluster/apps/application_1326821518301_0010" 
+
+Output:
+
+```json
+{
+   "app" : {
+      "finishedTime" : 0,
+      "amContainerLogs" : "http://host.domain.com:8042/node/containerlogs/container_1326821518301_0010_01_000001",
+      "trackingUI" : "ApplicationMaster",
+      "state" : "RUNNING",
+      "user" : "user1",
+      "id" : "application_1326821518301_0010",
+      "clusterId" : 1326821518301,
+      "finalStatus" : "UNDEFINED",
+      "amHostHttpAddress" : "host.domain.com:8042",
+      "progress" : 82.44703,
+      "name" : "Sleep job",
+      "startedTime" : 1326860715335,
+      "elapsedTime" : 31814,
+      "diagnostics" : "",
+      "trackingUrl" : "http://host.domain.com:8088/proxy/application_1326821518301_0010/",
+      "queue" : "a1"
+   }
+}
+```
+
+The user then wishes to get more details about the running application and goes directly to the MapReduce application master for this application. The ResourceManager lists the trackingUrl that can be used for this application: http://host.domain.com:8088/proxy/application\_1326821518301\_0010. This could either go to the web browser or use the web service REST API's. The user uses the web services REST API's to get the list of jobs this MapReduce application master is running:
+
+     curl --compressed -H "Accept: application/json" -X GET "http://host.domain.com:8088/proxy/application_1326821518301_0010/ws/v1/mapreduce/jobs"
+
+Output:
+
+```json
+{
+   "jobs" : {
+      "job" : [
+         {
+            "runningReduceAttempts" : 1,
+            "reduceProgress" : 72.104515,
+            "failedReduceAttempts" : 0,
+            "newMapAttempts" : 0,
+            "mapsRunning" : 0,
+            "state" : "RUNNING",
+            "successfulReduceAttempts" : 0,
+            "reducesRunning" : 1,
+            "acls" : [
+               {
+                  "value" : " ",
+                  "name" : "mapreduce.job.acl-modify-job"
+               },
+               {
+                  "value" : " ",
+                  "name" : "mapreduce.job.acl-view-job"
+               }
+            ],
+            "reducesPending" : 0,
+            "user" : "user1",
+            "reducesTotal" : 1,
+            "mapsCompleted" : 1,
+            "startTime" : 1326860720902,
+            "id" : "job_1326821518301_10_10",
+            "successfulMapAttempts" : 1,
+            "runningMapAttempts" : 0,
+            "newReduceAttempts" : 0,
+            "name" : "Sleep job",
+            "mapsPending" : 0,
+            "elapsedTime" : 64432,
+            "reducesCompleted" : 0,
+            "mapProgress" : 100,
+            "diagnostics" : "",
+            "failedMapAttempts" : 0,
+            "killedReduceAttempts" : 0,
+            "mapsTotal" : 1,
+            "uberized" : false,
+            "killedMapAttempts" : 0,
+            "finishTime" : 0
+         }
+      ]
+   }
+}
+```
+
+The user then wishes to get the task details about the job with job id job\_1326821518301\_10\_10 that was listed above.
+
+     curl --compressed -H "Accept: application/json" -X GET "http://host.domain.com:8088/proxy/application_1326821518301_0010/ws/v1/mapreduce/jobs/job_1326821518301_10_10/tasks" 
+
+Output:
+
+```json
+{
+   "tasks" : {
+      "task" : [
+         {
+            "progress" : 100,
+            "elapsedTime" : 5059,
+            "state" : "SUCCEEDED",
+            "startTime" : 1326860725014,
+            "id" : "task_1326821518301_10_10_m_0",
+            "type" : "MAP",
+            "successfulAttempt" : "attempt_1326821518301_10_10_m_0_0",
+            "finishTime" : 1326860730073
+         },
+         {
+            "progress" : 72.104515,
+            "elapsedTime" : 0,
+            "state" : "RUNNING",
+            "startTime" : 1326860732984,
+            "id" : "task_1326821518301_10_10_r_0",
+            "type" : "REDUCE",
+            "successfulAttempt" : "",
+            "finishTime" : 0
+         }
+      ]
+   }
+}
+```
+
+The map task has finished but the reduce task is still running. The users wishes to get the task attempt information for the reduce task task\_1326821518301\_10\_10\_r\_0, note that the Accept header isn't really required here since JSON is the default output format:
+
+      curl --compressed -X GET "http://host.domain.com:8088/proxy/application_1326821518301_0010/ws/v1/mapreduce/jobs/job_1326821518301_10_10/tasks/task_1326821518301_10_10_r_0/attempts"
+
+Output:
+
+```json
+{
+   "taskAttempts" : {
+      "taskAttempt" : [
+         {
+            "elapsedMergeTime" : 158,
+            "shuffleFinishTime" : 1326860735378,
+            "assignedContainerId" : "container_1326821518301_0010_01_000003",
+            "progress" : 72.104515,
+            "elapsedTime" : 0,
+            "state" : "RUNNING",
+            "elapsedShuffleTime" : 2394,
+            "mergeFinishTime" : 1326860735536,
+            "rack" : "/10.10.10.0",
+            "elapsedReduceTime" : 0,
+            "nodeHttpAddress" : "host.domain.com:8042",
+            "type" : "REDUCE",
+            "startTime" : 1326860732984,
+            "id" : "attempt_1326821518301_10_10_r_0_0",
+            "finishTime" : 0
+         }
+      ]
+   }
+}
+```
+
+The reduce attempt is still running and the user wishes to see the current counter values for that attempt:
+
+     curl --compressed -H "Accept: application/json"  -X GET "http://host.domain.com:8088/proxy/application_1326821518301_0010/ws/v1/mapreduce/jobs/job_1326821518301_10_10/tasks/task_1326821518301_10_10_r_0/attempts/attempt_1326821518301_10_10_r_0_0/counters" 
+
+Output:
+
+```json
+{
+   "JobTaskAttemptCounters" : {
+      "taskAttemptCounterGroup" : [
+         {
+            "counterGroupName" : "org.apache.hadoop.mapreduce.FileSystemCounter",
+            "counter" : [
+               {
+                  "value" : 4216,
+                  "name" : "FILE_BYTES_READ"
+               }, 
+               {
+                  "value" : 77151,
+                  "name" : "FILE_BYTES_WRITTEN"
+               }, 
+               {
+                  "value" : 0,
+                  "name" : "FILE_READ_OPS"
+               },
+               {
+                  "value" : 0,
+                  "name" : "FILE_LARGE_READ_OPS"
+               },
+               {
+                  "value" : 0,
+                  "name" : "FILE_WRITE_OPS"
+               },
+               {
+                  "value" : 0,
+                  "name" : "HDFS_BYTES_READ"
+               },
+               {
+                  "value" : 0,
+                  "name" : "HDFS_BYTES_WRITTEN"
+               },
+               {
+                  "value" : 0,
+                  "name" : "HDFS_READ_OPS"
+               },
+               {
+                  "value" : 0,
+                  "name" : "HDFS_LARGE_READ_OPS"
+               },
+               {
+                  "value" : 0,
+                  "name" : "HDFS_WRITE_OPS"
+               }
+            ]  
+         }, 
+         {
+            "counterGroupName" : "org.apache.hadoop.mapreduce.TaskCounter",
+            "counter" : [
+               {
+                  "value" : 0,
+                  "name" : "COMBINE_INPUT_RECORDS"
+               }, 
+               {
+                  "value" : 0,
+                  "name" : "COMBINE_OUTPUT_RECORDS"
+               }, 
+               {  
+                  "value" : 1767,
+                  "name" : "REDUCE_INPUT_GROUPS"
+               },
+               {  
+                  "value" : 25104,
+                  "name" : "REDUCE_SHUFFLE_BYTES"
+               },
+               {
+                  "value" : 1767,
+                  "name" : "REDUCE_INPUT_RECORDS"
+               },
+               {
+                  "value" : 0,
+                  "name" : "REDUCE_OUTPUT_RECORDS"
+               },
+               {
+                  "value" : 0,
+                  "name" : "SPILLED_RECORDS"
+               },
+               {
+                  "value" : 1,
+                  "name" : "SHUFFLED_MAPS"
+               },
+               {
+                  "value" : 0,
+                  "name" : "FAILED_SHUFFLE"
+               },
+               {
+                  "value" : 1,
+                  "name" : "MERGED_MAP_OUTPUTS"
+               },
+               {
+                  "value" : 50,
+                  "name" : "GC_TIME_MILLIS"
+               },
+               {
+                  "value" : 1580,
+                  "name" : "CPU_MILLISECONDS"
+               },
+               {
+                  "value" : 141320192,
+                  "name" : "PHYSICAL_MEMORY_BYTES"
+               },
+              {
+                  "value" : 1118552064,
+                  "name" : "VIRTUAL_MEMORY_BYTES"
+               }, 
+               {  
+                  "value" : 73728000,
+                  "name" : "COMMITTED_HEAP_BYTES"
+               }
+            ]
+         },
+         {  
+            "counterGroupName" : "Shuffle Errors",
+            "counter" : [
+               {  
+                  "value" : 0,
+                  "name" : "BAD_ID"
+               },
+               {  
+                  "value" : 0,
+                  "name" : "CONNECTION"
+               },
+               {  
+                  "value" : 0,
+                  "name" : "IO_ERROR"
+               },
+               {  
+                  "value" : 0,
+                  "name" : "WRONG_LENGTH"
+               },
+               {  
+                  "value" : 0,
+                  "name" : "WRONG_MAP"
+               },
+               {  
+                  "value" : 0,
+                  "name" : "WRONG_REDUCE"
+               }
+            ]
+         },
+         {  
+            "counterGroupName" : "org.apache.hadoop.mapreduce.lib.output.FileOutputFormatCounter",
+            "counter" : [
+              {  
+                  "value" : 0,
+                  "name" : "BYTES_WRITTEN"
+               }
+            ]
+         }
+      ],
+      "id" : "attempt_1326821518301_10_10_r_0_0"
+   }
+}
+```
+
+The job finishes and the user wishes to get the final job information from the history server for this job.
+
+      curl --compressed -X GET "http://host.domain.com:19888/ws/v1/history/mapreduce/jobs/job_1326821518301_10_10" 
+
+Output:
+
+```json
+{
+   "job" : {
+      "avgReduceTime" : 1250784,
+      "failedReduceAttempts" : 0,
+      "state" : "SUCCEEDED",
+      "successfulReduceAttempts" : 1,
+      "acls" : [
+         {
+            "value" : " ",
+            "name" : "mapreduce.job.acl-modify-job"
+         },
+         {
+            "value" : " ",
+            "name" : "mapreduce.job.acl-view-job"
+         }
+      ],
+      "user" : "user1",
+      "reducesTotal" : 1,
+      "mapsCompleted" : 1,
+      "startTime" : 1326860720902,
+      "id" : "job_1326821518301_10_10",
+      "avgMapTime" : 5059,
+      "successfulMapAttempts" : 1,
+      "name" : "Sleep job",
+      "avgShuffleTime" : 2394,
+      "reducesCompleted" : 1,
+      "diagnostics" : "",
+      "failedMapAttempts" : 0,
+      "avgMergeTime" : 2552,
+      "killedReduceAttempts" : 0,
+      "mapsTotal" : 1,
+      "queue" : "a1",
+      "uberized" : false,
+      "killedMapAttempts" : 0,
+      "finishTime" : 1326861986164
+   }
+}
+```
+
+The user also gets the final applications information from the ResourceManager.
+
+      curl --compressed -H "Accept: application/json" -X GET "http://host.domain.com:8088/ws/v1/cluster/apps/application_1326821518301_0010" 
+
+Output:
+
+```json
+{
+   "app" : {
+      "finishedTime" : 1326861991282,
+      "amContainerLogs" : "http://host.domain.com:8042/node/containerlogs/container_1326821518301_0010_01_000001",
+      "trackingUI" : "History",
+      "state" : "FINISHED",
+      "user" : "user1",
+      "id" : "application_1326821518301_0010",
+      "clusterId" : 1326821518301,
+      "finalStatus" : "SUCCEEDED",
+      "amHostHttpAddress" : "host.domain.com:8042",
+      "progress" : 100,
+      "name" : "Sleep job",
+      "startedTime" : 1326860715335,
+      "elapsedTime" : 1275947,
+      "diagnostics" : "",
+      "trackingUrl" : "http://host.domain.com:8088/proxy/application_1326821518301_0010/jobhistory/job/job_1326821518301_10_10",
+      "queue" : "a1"
+   }
+}
+```
\ No newline at end of file


[24/50] [abbrv] hadoop git commit: HADOOP-11569. Provide Merge API for MapFile to merge multiple similar MapFiles to one MapFile. Contributed by Vinayakumar B.

Posted by zh...@apache.org.
HADOOP-11569. Provide Merge API for MapFile to merge multiple similar MapFiles to one MapFile. 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/645ebb96
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/645ebb96
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/645ebb96

Branch: refs/heads/HDFS-7285
Commit: 645ebb965b88cb3018fb1588268cfaf8db837431
Parents: cc02446
Author: Tsuyoshi Ozawa <oz...@apache.org>
Authored: Fri Feb 27 17:46:07 2015 +0900
Committer: Zhe Zhang <zh...@cloudera.com>
Committed: Mon Mar 2 09:13:53 2015 -0800

----------------------------------------------------------------------
 hadoop-common-project/hadoop-common/CHANGES.txt |   3 +
 .../main/java/org/apache/hadoop/io/MapFile.java | 143 +++++++++++++++++++
 .../java/org/apache/hadoop/io/TestMapFile.java  |  56 ++++++++
 3 files changed, 202 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/645ebb96/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 1d9a6d4..6d4da77 100644
--- a/hadoop-common-project/hadoop-common/CHANGES.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES.txt
@@ -445,6 +445,9 @@ Release 2.7.0 - UNRELEASED
 
     HADOOP-11510. Expose truncate API via FileContext. (yliu)
 
+    HADOOP-11569. Provide Merge API for MapFile to merge multiple similar MapFiles
+    to one MapFile. (Vinayakumar B via ozawa)
+
   IMPROVEMENTS
 
     HADOOP-11483. HardLink.java should use the jdk7 createLink method (aajisaka)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/645ebb96/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/MapFile.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/MapFile.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/MapFile.java
index 84c9dcc..ee76458 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/MapFile.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/MapFile.java
@@ -25,6 +25,7 @@ import java.util.Arrays;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.HadoopIllegalArgumentException;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
@@ -824,6 +825,148 @@ public class MapFile {
     return cnt;
   }
 
+  /**
+   * Class to merge multiple MapFiles of same Key and Value types to one MapFile
+   */
+  public static class Merger {
+    private Configuration conf;
+    private WritableComparator comparator = null;
+    private Reader[] inReaders;
+    private Writer outWriter;
+    private Class<Writable> valueClass = null;
+    private Class<WritableComparable> keyClass = null;
+
+    public Merger(Configuration conf) throws IOException {
+      this.conf = conf;
+    }
+
+    /**
+     * Merge multiple MapFiles to one Mapfile
+     *
+     * @param inMapFiles
+     * @param outMapFile
+     * @throws IOException
+     */
+    public void merge(Path[] inMapFiles, boolean deleteInputs,
+        Path outMapFile) throws IOException {
+      try {
+        open(inMapFiles, outMapFile);
+        mergePass();
+      } finally {
+        close();
+      }
+      if (deleteInputs) {
+        for (int i = 0; i < inMapFiles.length; i++) {
+          Path path = inMapFiles[i];
+          delete(path.getFileSystem(conf), path.toString());
+        }
+      }
+    }
+
+    /*
+     * Open all input files for reading and verify the key and value types. And
+     * open Output file for writing
+     */
+    @SuppressWarnings("unchecked")
+    private void open(Path[] inMapFiles, Path outMapFile) throws IOException {
+      inReaders = new Reader[inMapFiles.length];
+      for (int i = 0; i < inMapFiles.length; i++) {
+        Reader reader = new Reader(inMapFiles[i], conf);
+        if (keyClass == null || valueClass == null) {
+          keyClass = (Class<WritableComparable>) reader.getKeyClass();
+          valueClass = (Class<Writable>) reader.getValueClass();
+        } else if (keyClass != reader.getKeyClass()
+            || valueClass != reader.getValueClass()) {
+          throw new HadoopIllegalArgumentException(
+              "Input files cannot be merged as they"
+                  + " have different Key and Value classes");
+        }
+        inReaders[i] = reader;
+      }
+
+      if (comparator == null) {
+        Class<? extends WritableComparable> cls;
+        cls = keyClass.asSubclass(WritableComparable.class);
+        this.comparator = WritableComparator.get(cls, conf);
+      } else if (comparator.getKeyClass() != keyClass) {
+        throw new HadoopIllegalArgumentException(
+            "Input files cannot be merged as they"
+                + " have different Key class compared to"
+                + " specified comparator");
+      }
+
+      outWriter = new MapFile.Writer(conf, outMapFile,
+          MapFile.Writer.keyClass(keyClass),
+          MapFile.Writer.valueClass(valueClass));
+    }
+
+    /**
+     * Merge all input files to output map file.<br>
+     * 1. Read first key/value from all input files to keys/values array. <br>
+     * 2. Select the least key and corresponding value. <br>
+     * 3. Write the selected key and value to output file. <br>
+     * 4. Replace the already written key/value in keys/values arrays with the
+     * next key/value from the selected input <br>
+     * 5. Repeat step 2-4 till all keys are read. <br>
+     */
+    private void mergePass() throws IOException {
+      // re-usable array
+      WritableComparable[] keys = new WritableComparable[inReaders.length];
+      Writable[] values = new Writable[inReaders.length];
+      // Read first key/value from all inputs
+      for (int i = 0; i < inReaders.length; i++) {
+        keys[i] = ReflectionUtils.newInstance(keyClass, null);
+        values[i] = ReflectionUtils.newInstance(valueClass, null);
+        if (!inReaders[i].next(keys[i], values[i])) {
+          // Handle empty files
+          keys[i] = null;
+          values[i] = null;
+        }
+      }
+
+      do {
+        int currentEntry = -1;
+        WritableComparable currentKey = null;
+        Writable currentValue = null;
+        for (int i = 0; i < keys.length; i++) {
+          if (keys[i] == null) {
+            // Skip Readers reached EOF
+            continue;
+          }
+          if (currentKey == null || comparator.compare(currentKey, keys[i]) > 0) {
+            currentEntry = i;
+            currentKey = keys[i];
+            currentValue = values[i];
+          }
+        }
+        if (currentKey == null) {
+          // Merge Complete
+          break;
+        }
+        // Write the selected key/value to merge stream
+        outWriter.append(currentKey, currentValue);
+        // Replace the already written key/value in keys/values arrays with the
+        // next key/value from the selected input
+        if (!inReaders[currentEntry].next(keys[currentEntry],
+            values[currentEntry])) {
+          // EOF for this file
+          keys[currentEntry] = null;
+          values[currentEntry] = null;
+        }
+      } while (true);
+    }
+
+    private void close() throws IOException {
+      for (int i = 0; i < inReaders.length; i++) {
+        IOUtils.closeStream(inReaders[i]);
+        inReaders[i] = null;
+      }
+      if (outWriter != null) {
+        outWriter.close();
+        outWriter = null;
+      }
+    }
+  }
 
   public static void main(String[] args) throws Exception {
     String usage = "Usage: MapFile inFile outFile";

http://git-wip-us.apache.org/repos/asf/hadoop/blob/645ebb96/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/TestMapFile.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/TestMapFile.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/TestMapFile.java
index ced74fb..3f14de0 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/TestMapFile.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/TestMapFile.java
@@ -21,6 +21,10 @@ import java.io.File;
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.OutputStream;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
@@ -730,4 +734,56 @@ public class TestMapFile {
       reader.close();
     }
   }
+
+  @Test
+  public void testMerge() throws Exception {
+    final String TEST_METHOD_KEY = "testMerge.mapfile";
+    int SIZE = 10;
+    int ITERATIONS = 5;
+    Path[] in = new Path[5];
+    List<Integer> expected = new ArrayList<Integer>();
+    for (int j = 0; j < 5; j++) {
+      try (MapFile.Writer writer = createWriter(TEST_METHOD_KEY + "." + j,
+          IntWritable.class, Text.class)) {
+        in[j] = new Path(TEST_DIR, TEST_METHOD_KEY + "." + j);
+        for (int i = 0; i < SIZE; i++) {
+          expected.add(i + j);
+          writer.append(new IntWritable(i + j), new Text("Value:" + (i + j)));
+        }
+      }
+    }
+    // Sort expected values
+    Collections.sort(expected);
+    // Merge all 5 files
+    MapFile.Merger merger = new MapFile.Merger(conf);
+    merger.merge(in, true, new Path(TEST_DIR, TEST_METHOD_KEY));
+
+    try (MapFile.Reader reader = createReader(TEST_METHOD_KEY,
+        IntWritable.class)) {
+      int start = 0;
+      // test iteration
+      Text startValue = new Text("Value:" + start);
+      int i = 0;
+      while (i++ < ITERATIONS) {
+        Iterator<Integer> expectedIterator = expected.iterator();
+        IntWritable key = new IntWritable(start);
+        Text value = startValue;
+        IntWritable prev = new IntWritable(start);
+        while (reader.next(key, value)) {
+          assertTrue("Next key should be always equal or more",
+              prev.get() <= key.get());
+          assertEquals(expectedIterator.next().intValue(), key.get());
+          prev.set(key.get());
+        }
+        reader.reset();
+      }
+    }
+
+    // inputs should be deleted
+    for (int j = 0; j < in.length; j++) {
+      Path path = in[j];
+      assertFalse("inputs should be deleted",
+          path.getFileSystem(conf).exists(path));
+    }
+  }
 }


[29/50] [abbrv] hadoop git commit: YARN-3168. Convert site documentation from apt to markdown (Gururaj Shetty via aw)

Posted by zh...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/06aca7c6/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/WritingYarnApplications.md
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/WritingYarnApplications.md b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/WritingYarnApplications.md
new file mode 100644
index 0000000..5e4df9f
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/WritingYarnApplications.md
@@ -0,0 +1,591 @@
+<!---
+  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: Writing YARN Applications
+=================================
+
+* [Purpose](#Purpose)
+* [Concepts and Flow](#Concepts_and_Flow)
+* [Interfaces](#Interfaces)
+* [Writing a Simple Yarn Application](#Writing_a_Simple_Yarn_Application)
+    * [Writing a simple Client](#Writing_a_simple_Client)
+    * [Writing an ApplicationMaster (AM)](#Writing_an_ApplicationMaster_AM)
+* [FAQ](#FAQ)
+    * [How can I distribute my application's jars to all of the nodes in the YARN cluster that need it?](#How_can_I_distribute_my_applications_jars_to_all_of_the_nodes_in_the_YARN_cluster_that_need_it)
+    * [How do I get the ApplicationMaster's ApplicationAttemptId?](#How_do_I_get_the_ApplicationMasters_ApplicationAttemptId)
+    * [Why my container is killed by the NodeManager?](#Why_my_container_is_killed_by_the_NodeManager)
+    * [How do I include native libraries?](#How_do_I_include_native_libraries)
+* [Useful Links](#Useful_Links)
+* [Sample Code](#Sample_Code)
+
+Purpose
+-------
+
+This document describes, at a high-level, the way to implement new Applications for YARN.
+
+Concepts and Flow
+-----------------
+
+The general concept is that an *application submission client* submits an *application* to the YARN *ResourceManager* (RM). This can be done through setting up a `YarnClient` object. After `YarnClient` is started, the client can then set up application context, prepare the very first container of the application that contains the *ApplicationMaster* (AM), and then submit the application. You need to provide information such as the details about the local files/jars that need to be available for your application to run, the actual command that needs to be executed (with the necessary command line arguments), any OS environment settings (optional), etc. Effectively, you need to describe the Unix process(es) that needs to be launched for your ApplicationMaster.
+
+The YARN ResourceManager will then launch the ApplicationMaster (as specified) on an allocated container. The ApplicationMaster communicates with YARN cluster, and handles application execution. It performs operations in an asynchronous fashion. During application launch time, the main tasks of the ApplicationMaster are: a) communicating with the ResourceManager to negotiate and allocate resources for future containers, and b) after container allocation, communicating YARN *NodeManager*s (NMs) to launch application containers on them. Task a) can be performed asynchronously through an `AMRMClientAsync` object, with event handling methods specified in a `AMRMClientAsync.CallbackHandler` type of event handler. The event handler needs to be set to the client explicitly. Task b) can be performed by launching a runnable object that then launches containers when there are containers allocated. As part of launching this container, the AM has to specify the `ContainerLaunchContext` that has
  the launch information such as command line specification, environment, etc.
+
+During the execution of an application, the ApplicationMaster communicates NodeManagers through `NMClientAsync` object. All container events are handled by `NMClientAsync.CallbackHandler`, associated with `NMClientAsync`. A typical callback handler handles client start, stop, status update and error. ApplicationMaster also reports execution progress to ResourceManager by handling the `getProgress()` method of `AMRMClientAsync.CallbackHandler`.
+
+Other than asynchronous clients, there are synchronous versions for certain workflows (`AMRMClient` and `NMClient`). The asynchronous clients are recommended because of (subjectively) simpler usages, and this article will mainly cover the asynchronous clients. Please refer to `AMRMClient` and `NMClient` for more information on synchronous clients.
+
+Interfaces
+----------
+
+Following are the important interfaces:
+
+* **Client**\<-\->**ResourceManager** 
+    
+    By using `YarnClient` objects.
+
+* **ApplicationMaster**\<-\->**ResourceManager**
+
+    By using `AMRMClientAsync` objects, handling events asynchronously by `AMRMClientAsync.CallbackHandler`
+
+* **ApplicationMaster**\<-\->**NodeManager**
+
+    Launch containers. Communicate with NodeManagers by using `NMClientAsync` objects, handling container events by `NMClientAsync.CallbackHandler`
+
+**Note**
+
+* The three main protocols for YARN application (ApplicationClientProtocol, ApplicationMasterProtocol and ContainerManagementProtocol) are still preserved. The 3 clients wrap these 3 protocols to provide simpler programming model for YARN applications.
+
+* Under very rare circumstances, programmer may want to directly use the 3 protocols to implement an application. However, note that *such behaviors are no longer encouraged for general use cases*.
+
+Writing a Simple Yarn Application
+---------------------------------
+
+### Writing a simple Client
+
+* The first step that a client needs to do is to initialize and start a YarnClient.
+
+          YarnClient yarnClient = YarnClient.createYarnClient();
+          yarnClient.init(conf);
+          yarnClient.start();
+
+* Once a client is set up, the client needs to create an application, and get its application id.
+
+          YarnClientApplication app = yarnClient.createApplication();
+          GetNewApplicationResponse appResponse = app.getNewApplicationResponse();
+
+* The response from the `YarnClientApplication` for a new application also contains information about the cluster such as the minimum/maximum resource capabilities of the cluster. This is required so that to ensure that you can correctly set the specifications of the container in which the ApplicationMaster would be launched. Please refer to `GetNewApplicationResponse` for more details.
+
+* The main crux of a client is to setup the `ApplicationSubmissionContext` which defines all the information needed by the RM to launch the AM. A client needs to set the following into the context:
+
+  * Application info: id, name
+
+  * Queue, priority info: Queue to which the application will be submitted, the priority to be assigned for the application.
+
+  * User: The user submitting the application
+
+  * `ContainerLaunchContext`: The information defining the container in which the AM will be launched and run. The `ContainerLaunchContext`, as mentioned previously, defines all the required information needed to run the application such as the local **R**esources (binaries, jars, files etc.), **E**nvironment settings (CLASSPATH etc.), the **C**ommand to be executed and security **T**okens (*RECT*).
+
+```java
+// set the application submission context
+ApplicationSubmissionContext appContext = app.getApplicationSubmissionContext();
+ApplicationId appId = appContext.getApplicationId();
+
+appContext.setKeepContainersAcrossApplicationAttempts(keepContainers);
+appContext.setApplicationName(appName);
+
+// set local resources for the application master
+// local files or archives as needed
+// In this scenario, the jar file for the application master is part of the local resources
+Map<String, LocalResource> localResources = new HashMap<String, LocalResource>();
+
+LOG.info("Copy App Master jar from local filesystem and add to local environment");
+// Copy the application master jar to the filesystem
+// Create a local resource to point to the destination jar path
+FileSystem fs = FileSystem.get(conf);
+addToLocalResources(fs, appMasterJar, appMasterJarPath, appId.toString(),
+    localResources, null);
+
+// Set the log4j properties if needed
+if (!log4jPropFile.isEmpty()) {
+  addToLocalResources(fs, log4jPropFile, log4jPath, appId.toString(),
+      localResources, null);
+}
+
+// The shell script has to be made available on the final container(s)
+// where it will be executed.
+// To do this, we need to first copy into the filesystem that is visible
+// to the yarn framework.
+// We do not need to set this as a local resource for the application
+// master as the application master does not need it.
+String hdfsShellScriptLocation = "";
+long hdfsShellScriptLen = 0;
+long hdfsShellScriptTimestamp = 0;
+if (!shellScriptPath.isEmpty()) {
+  Path shellSrc = new Path(shellScriptPath);
+  String shellPathSuffix =
+      appName + "/" + appId.toString() + "/" + SCRIPT_PATH;
+  Path shellDst =
+      new Path(fs.getHomeDirectory(), shellPathSuffix);
+  fs.copyFromLocalFile(false, true, shellSrc, shellDst);
+  hdfsShellScriptLocation = shellDst.toUri().toString();
+  FileStatus shellFileStatus = fs.getFileStatus(shellDst);
+  hdfsShellScriptLen = shellFileStatus.getLen();
+  hdfsShellScriptTimestamp = shellFileStatus.getModificationTime();
+}
+
+if (!shellCommand.isEmpty()) {
+  addToLocalResources(fs, null, shellCommandPath, appId.toString(),
+      localResources, shellCommand);
+}
+
+if (shellArgs.length > 0) {
+  addToLocalResources(fs, null, shellArgsPath, appId.toString(),
+      localResources, StringUtils.join(shellArgs, " "));
+}
+
+// Set the env variables to be setup in the env where the application master will be run
+LOG.info("Set the environment for the application master");
+Map<String, String> env = new HashMap<String, String>();
+
+// put location of shell script into env
+// using the env info, the application master will create the correct local resource for the
+// eventual containers that will be launched to execute the shell scripts
+env.put(DSConstants.DISTRIBUTEDSHELLSCRIPTLOCATION, hdfsShellScriptLocation);
+env.put(DSConstants.DISTRIBUTEDSHELLSCRIPTTIMESTAMP, Long.toString(hdfsShellScriptTimestamp));
+env.put(DSConstants.DISTRIBUTEDSHELLSCRIPTLEN, Long.toString(hdfsShellScriptLen));
+
+// Add AppMaster.jar location to classpath
+// At some point we should not be required to add
+// the hadoop specific classpaths to the env.
+// It should be provided out of the box.
+// For now setting all required classpaths including
+// the classpath to "." for the application jar
+StringBuilder classPathEnv = new StringBuilder(Environment.CLASSPATH.$$())
+  .append(ApplicationConstants.CLASS_PATH_SEPARATOR).append("./*");
+for (String c : conf.getStrings(
+    YarnConfiguration.YARN_APPLICATION_CLASSPATH,
+    YarnConfiguration.DEFAULT_YARN_CROSS_PLATFORM_APPLICATION_CLASSPATH)) {
+  classPathEnv.append(ApplicationConstants.CLASS_PATH_SEPARATOR);
+  classPathEnv.append(c.trim());
+}
+classPathEnv.append(ApplicationConstants.CLASS_PATH_SEPARATOR).append(
+  "./log4j.properties");
+
+// Set the necessary command to execute the application master
+Vector<CharSequence> vargs = new Vector<CharSequence>(30);
+
+// Set java executable command
+LOG.info("Setting up app master command");
+vargs.add(Environment.JAVA_HOME.$$() + "/bin/java");
+// Set Xmx based on am memory size
+vargs.add("-Xmx" + amMemory + "m");
+// Set class name
+vargs.add(appMasterMainClass);
+// Set params for Application Master
+vargs.add("--container_memory " + String.valueOf(containerMemory));
+vargs.add("--container_vcores " + String.valueOf(containerVirtualCores));
+vargs.add("--num_containers " + String.valueOf(numContainers));
+vargs.add("--priority " + String.valueOf(shellCmdPriority));
+
+for (Map.Entry<String, String> entry : shellEnv.entrySet()) {
+  vargs.add("--shell_env " + entry.getKey() + "=" + entry.getValue());
+}
+if (debugFlag) {
+  vargs.add("--debug");
+}
+
+vargs.add("1>" + ApplicationConstants.LOG_DIR_EXPANSION_VAR + "/AppMaster.stdout");
+vargs.add("2>" + ApplicationConstants.LOG_DIR_EXPANSION_VAR + "/AppMaster.stderr");
+
+// Get final commmand
+StringBuilder command = new StringBuilder();
+for (CharSequence str : vargs) {
+  command.append(str).append(" ");
+}
+
+LOG.info("Completed setting up app master command " + command.toString());
+List<String> commands = new ArrayList<String>();
+commands.add(command.toString());
+
+// Set up the container launch context for the application master
+ContainerLaunchContext amContainer = ContainerLaunchContext.newInstance(
+  localResources, env, commands, null, null, null);
+
+// Set up resource type requirements
+// For now, both memory and vcores are supported, so we set memory and
+// vcores requirements
+Resource capability = Resource.newInstance(amMemory, amVCores);
+appContext.setResource(capability);
+
+// Service data is a binary blob that can be passed to the application
+// Not needed in this scenario
+// amContainer.setServiceData(serviceData);
+
+// Setup security tokens
+if (UserGroupInformation.isSecurityEnabled()) {
+  // Note: Credentials class is marked as LimitedPrivate for HDFS and MapReduce
+  Credentials credentials = new Credentials();
+  String tokenRenewer = conf.get(YarnConfiguration.RM_PRINCIPAL);
+  if (tokenRenewer == null | | tokenRenewer.length() == 0) {
+    throw new IOException(
+      "Can't get Master Kerberos principal for the RM to use as renewer");
+  }
+
+  // For now, only getting tokens for the default file-system.
+  final Token<?> tokens[] =
+      fs.addDelegationTokens(tokenRenewer, credentials);
+  if (tokens != null) {
+    for (Token<?> token : tokens) {
+      LOG.info("Got dt for " + fs.getUri() + "; " + token);
+    }
+  }
+  DataOutputBuffer dob = new DataOutputBuffer();
+  credentials.writeTokenStorageToStream(dob);
+  ByteBuffer fsTokens = ByteBuffer.wrap(dob.getData(), 0, dob.getLength());
+  amContainer.setTokens(fsTokens);
+}
+
+appContext.setAMContainerSpec(amContainer);
+```
+
+* After the setup process is complete, the client is ready to submit the application with specified priority and queue.
+
+```java
+// Set the priority for the application master
+Priority pri = Priority.newInstance(amPriority);
+appContext.setPriority(pri);
+
+// Set the queue to which this application is to be submitted in the RM
+appContext.setQueue(amQueue);
+
+// Submit the application to the applications manager
+// SubmitApplicationResponse submitResp = applicationsManager.submitApplication(appRequest);
+
+yarnClient.submitApplication(appContext);
+```
+
+* At this point, the RM will have accepted the application and in the background, will go through the process of allocating a container with the required specifications and then eventually setting up and launching the AM on the allocated container.
+
+* There are multiple ways a client can track progress of the actual task.
+   
+> * It can communicate with the RM and request for a report of the application via the `getApplicationReport()` method of `YarnClient`.
+
+```java
+// Get application report for the appId we are interested in
+ApplicationReport report = yarnClient.getApplicationReport(appId);
+```
+  
+> The ApplicationReport received from the RM consists of the following:
+
+>> * *General application information*: Application id, queue to which the application was submitted, user who submitted the application and the start time for the application.
+
+>> * *ApplicationMaster details*: the host on which the AM is running, the rpc port (if any) on which it is listening for requests from clients and a token that the client needs to communicate with the AM.
+
+>> * *Application tracking information*: If the application supports some form of progress tracking, it can set a tracking url which is available via `ApplicationReport`'s `getTrackingUrl()` method that a client can look at to monitor progress.
+
+>> * *Application status*: The state of the application as seen by the ResourceManager is available via `ApplicationReport#getYarnApplicationState`. If the `YarnApplicationState` is set to `FINISHED`, the client should refer to `ApplicationReport#getFinalApplicationStatus` to check for the actual success/failure of the application task itself. In case of failures, `ApplicationReport#getDiagnostics` may be useful to shed some more light on the the failure.
+
+> * If the ApplicationMaster supports it, a client can directly query the AM itself for progress updates via the host:rpcport information obtained from the application report. It can also use the tracking url obtained from the report if available.
+
+* In certain situations, if the application is taking too long or due to other factors, the client may wish to kill the application. `YarnClient` supports the `killApplication` call that allows a client to send a kill signal to the AM via the ResourceManager. An ApplicationMaster if so designed may also support an abort call via its rpc layer that a client may be able to leverage.
+
+          yarnClient.killApplication(appId);
+
+### Writing an ApplicationMaster (AM)
+
+* The AM is the actual owner of the job. It will be launched by the RM and via the client will be provided all the necessary information and resources about the job that it has been tasked with to oversee and complete.
+
+* As the AM is launched within a container that may (likely will) be sharing a physical host with other containers, given the multi-tenancy nature, amongst other issues, it cannot make any assumptions of things like pre-configured ports that it can listen on.
+
+* When the AM starts up, several parameters are made available to it via the environment. These include the `ContainerId` for the AM container, the application submission time and details about the NM (NodeManager) host running the ApplicationMaster. Ref `ApplicationConstants` for parameter names.
+
+* All interactions with the RM require an `ApplicationAttemptId` (there can be multiple attempts per application in case of failures). The `ApplicationAttemptId` can be obtained from the AM's container id. There are helper APIs to convert the value obtained from the environment into objects.
+
+```java
+Map<String, String> envs = System.getenv();
+String containerIdString =
+    envs.get(ApplicationConstants.AM_CONTAINER_ID_ENV);
+if (containerIdString == null) {
+  // container id should always be set in the env by the framework
+  throw new IllegalArgumentException(
+      "ContainerId not set in the environment");
+}
+ContainerId containerId = ConverterUtils.toContainerId(containerIdString);
+ApplicationAttemptId appAttemptID = containerId.getApplicationAttemptId();
+```
+
+* After an AM has initialized itself completely, we can start the two clients: one to ResourceManager, and one to NodeManagers. We set them up with our customized event handler, and we will talk about those event handlers in detail later in this article.
+
+```java
+  AMRMClientAsync.CallbackHandler allocListener = new RMCallbackHandler();
+  amRMClient = AMRMClientAsync.createAMRMClientAsync(1000, allocListener);
+  amRMClient.init(conf);
+  amRMClient.start();
+
+  containerListener = createNMCallbackHandler();
+  nmClientAsync = new NMClientAsyncImpl(containerListener);
+  nmClientAsync.init(conf);
+  nmClientAsync.start();
+```
+
+* The AM has to emit heartbeats to the RM to keep it informed that the AM is alive and still running. The timeout expiry interval at the RM is defined by a config setting accessible via `YarnConfiguration.RM_AM_EXPIRY_INTERVAL_MS` with the default being defined by `YarnConfiguration.DEFAULT_RM_AM_EXPIRY_INTERVAL_MS`. The ApplicationMaster needs to register itself with the ResourceManager to start hearbeating.
+
+```java
+// Register self with ResourceManager
+// This will start heartbeating to the RM
+appMasterHostname = NetUtils.getHostname();
+RegisterApplicationMasterResponse response = amRMClient
+    .registerApplicationMaster(appMasterHostname, appMasterRpcPort,
+        appMasterTrackingUrl);
+```
+
+* In the response of the registration, maximum resource capability if included. You may want to use this to check the application's request.
+
+```java
+// Dump out information about cluster capability as seen by the
+// resource manager
+int maxMem = response.getMaximumResourceCapability().getMemory();
+LOG.info("Max mem capabililty of resources in this cluster " + maxMem);
+
+int maxVCores = response.getMaximumResourceCapability().getVirtualCores();
+LOG.info("Max vcores capabililty of resources in this cluster " + maxVCores);
+
+// A resource ask cannot exceed the max.
+if (containerMemory > maxMem) {
+  LOG.info("Container memory specified above max threshold of cluster."
+      + " Using max value." + ", specified=" + containerMemory + ", max="
+      + maxMem);
+  containerMemory = maxMem;
+}
+
+if (containerVirtualCores > maxVCores) {
+  LOG.info("Container virtual cores specified above max threshold of  cluster."
+    + " Using max value." + ", specified=" + containerVirtualCores + ", max="
+    + maxVCores);
+  containerVirtualCores = maxVCores;
+}
+List<Container> previousAMRunningContainers =
+    response.getContainersFromPreviousAttempts();
+LOG.info("Received " + previousAMRunningContainers.size()
+        + " previous AM's running containers on AM registration.");
+```
+
+* Based on the task requirements, the AM can ask for a set of containers to run its tasks on. We can now calculate how many containers we need, and request those many containers.
+
+```java
+List<Container> previousAMRunningContainers =
+    response.getContainersFromPreviousAttempts();
+List<Container> previousAMRunningContainers =
+    response.getContainersFromPreviousAttempts();
+LOG.info("Received " + previousAMRunningContainers.size()
+    + " previous AM's running containers on AM registration.");
+
+int numTotalContainersToRequest =
+    numTotalContainers - previousAMRunningContainers.size();
+// Setup ask for containers from RM
+// Send request for containers to RM
+// Until we get our fully allocated quota, we keep on polling RM for
+// containers
+// Keep looping until all the containers are launched and shell script
+// executed on them ( regardless of success/failure).
+for (int i = 0; i < numTotalContainersToRequest; ++i) {
+  ContainerRequest containerAsk = setupContainerAskForRM();
+  amRMClient.addContainerRequest(containerAsk);
+}
+```
+
+* In `setupContainerAskForRM()`, the follow two things need some set up:
+
+> * Resource capability: Currently, YARN supports memory based resource requirements so the request should define how much memory is needed. The value is defined in MB and has to less than the max capability of the cluster and an exact multiple of the min capability. Memory resources correspond to physical memory limits imposed on the task containers. It will also support computation based resource (vCore), as shown in the code.
+
+> * Priority: When asking for sets of containers, an AM may define different priorities to each set. For example, the Map-Reduce AM may assign a higher priority to containers needed for the Map tasks and a lower priority for the Reduce tasks' containers.
+
+```java
+private ContainerRequest setupContainerAskForRM() {
+  // setup requirements for hosts
+  // using * as any host will do for the distributed shell app
+  // set the priority for the request
+  Priority pri = Priority.newInstance(requestPriority);
+
+  // Set up resource type requirements
+  // For now, memory and CPU are supported so we set memory and cpu requirements
+  Resource capability = Resource.newInstance(containerMemory,
+    containerVirtualCores);
+
+  ContainerRequest request = new ContainerRequest(capability, null, null,
+      pri);
+  LOG.info("Requested container ask: " + request.toString());
+  return request;
+}
+```
+
+* After container allocation requests have been sent by the application manager, contailers will be launched asynchronously, by the event handler of the `AMRMClientAsync` client. The handler should implement `AMRMClientAsync.CallbackHandler` interface.
+
+> * When there are containers allocated, the handler sets up a thread that runs the code to launch containers. Here we use the name `LaunchContainerRunnable` to demonstrate. We will talk about the `LaunchContainerRunnable` class in the following part of this article.
+
+```java
+@Override
+public void onContainersAllocated(List<Container> allocatedContainers) {
+  LOG.info("Got response from RM for container ask, allocatedCnt="
+      + allocatedContainers.size());
+  numAllocatedContainers.addAndGet(allocatedContainers.size());
+  for (Container allocatedContainer : allocatedContainers) {
+    LaunchContainerRunnable runnableLaunchContainer =
+        new LaunchContainerRunnable(allocatedContainer, containerListener);
+    Thread launchThread = new Thread(runnableLaunchContainer);
+
+    // launch and start the container on a separate thread to keep
+    // the main thread unblocked
+    // as all containers may not be allocated at one go.
+    launchThreads.add(launchThread);
+    launchThread.start();
+  }
+}
+```
+
+> * On heart beat, the event handler reports the progress of the application.
+
+```java
+@Override
+public float getProgress() {
+  // set progress to deliver to RM on next heartbeat
+  float progress = (float) numCompletedContainers.get()
+      / numTotalContainers;
+  return progress;
+}
+```
+
+* The container launch thread actually launches the containers on NMs. After a container has been allocated to the AM, it needs to follow a similar process that the client followed in setting up the `ContainerLaunchContext` for the eventual task that is going to be running on the allocated Container. Once the `ContainerLaunchContext` is defined, the AM can start it through the `NMClientAsync`.
+
+```java
+// Set the necessary command to execute on the allocated container
+Vector<CharSequence> vargs = new Vector<CharSequence>(5);
+
+// Set executable command
+vargs.add(shellCommand);
+// Set shell script path
+if (!scriptPath.isEmpty()) {
+  vargs.add(Shell.WINDOWS ? ExecBatScripStringtPath
+    : ExecShellStringPath);
+}
+
+// Set args for the shell command if any
+vargs.add(shellArgs);
+// Add log redirect params
+vargs.add("1>" + ApplicationConstants.LOG_DIR_EXPANSION_VAR + "/stdout");
+vargs.add("2>" + ApplicationConstants.LOG_DIR_EXPANSION_VAR + "/stderr");
+
+// Get final commmand
+StringBuilder command = new StringBuilder();
+for (CharSequence str : vargs) {
+  command.append(str).append(" ");
+}
+
+List<String> commands = new ArrayList<String>();
+commands.add(command.toString());
+
+// Set up ContainerLaunchContext, setting local resource, environment,
+// command and token for constructor.
+
+// Note for tokens: Set up tokens for the container too. Today, for normal
+// shell commands, the container in distribute-shell doesn't need any
+// tokens. We are populating them mainly for NodeManagers to be able to
+// download anyfiles in the distributed file-system. The tokens are
+// otherwise also useful in cases, for e.g., when one is running a
+// "hadoop dfs" command inside the distributed shell.
+ContainerLaunchContext ctx = ContainerLaunchContext.newInstance(
+  localResources, shellEnv, commands, null, allTokens.duplicate(), null);
+containerListener.addContainer(container.getId(), container);
+nmClientAsync.startContainerAsync(container, ctx);
+```
+
+* The `NMClientAsync` object, together with its event handler, handles container events. Including container start, stop, status update, and occurs an error.
+
+* After the ApplicationMaster determines the work is done, it needs to unregister itself through the AM-RM client, and then stops the client.
+
+```java
+try {
+  amRMClient.unregisterApplicationMaster(appStatus, appMessage, null);
+} catch (YarnException ex) {
+  LOG.error("Failed to unregister application", ex);
+} catch (IOException e) {
+  LOG.error("Failed to unregister application", e);
+}
+
+amRMClient.stop();
+```
+
+FAQ
+---
+
+### How can I distribute my application's jars to all of the nodes in the YARN cluster that need it?
+
+You can use the LocalResource to add resources to your application request. This will cause YARN to distribute the resource to the ApplicationMaster node. If the resource is a tgz, zip, or jar - you can have YARN unzip it. Then, all you need to do is add the unzipped folder to your classpath. For example, when creating your application request:
+
+```java
+File packageFile = new File(packagePath);
+Url packageUrl = ConverterUtils.getYarnUrlFromPath(
+    FileContext.getFileContext.makeQualified(new Path(packagePath)));
+
+packageResource.setResource(packageUrl);
+packageResource.setSize(packageFile.length());
+packageResource.setTimestamp(packageFile.lastModified());
+packageResource.setType(LocalResourceType.ARCHIVE);
+packageResource.setVisibility(LocalResourceVisibility.APPLICATION);
+
+resource.setMemory(memory);
+containerCtx.setResource(resource);
+containerCtx.setCommands(ImmutableList.of(
+    "java -cp './package/*' some.class.to.Run "
+    + "1>" + ApplicationConstants.LOG_DIR_EXPANSION_VAR + "/stdout "
+    + "2>" + ApplicationConstants.LOG_DIR_EXPANSION_VAR + "/stderr"));
+containerCtx.setLocalResources(
+    Collections.singletonMap("package", packageResource));
+appCtx.setApplicationId(appId);
+appCtx.setUser(user.getShortUserName);
+appCtx.setAMContainerSpec(containerCtx);
+yarnClient.submitApplication(appCtx);
+```
+
+  As you can see, the `setLocalResources` command takes a map of names to resources. The name becomes a sym link in your application's cwd, so you can just refer to the artifacts inside by using ./package/\*.
+
+  **Note**: Java's classpath (cp) argument is VERY sensitive. Make sure you get the syntax EXACTLY correct.
+
+  Once your package is distributed to your AM, you'll need to follow the same process whenever your AM starts a new container (assuming you want the resources to be sent to your container). The code for this is the same. You just need to make sure that you give your AM the package path (either HDFS, or local), so that it can send the resource URL along with the container ctx.
+
+### How do I get the ApplicationMaster's `ApplicationAttemptId`?
+
+The `ApplicationAttemptId` will be passed to the AM via the environment and the value from the environment can be converted into an `ApplicationAttemptId` object via the ConverterUtils helper function.
+
+### Why my container is killed by the NodeManager?
+
+This is likely due to high memory usage exceeding your requested container memory size. There are a number of reasons that can cause this. First, look at the process tree that the NodeManager dumps when it kills your container. The two things you're interested in are physical memory and virtual memory. If you have exceeded physical memory limits your app is using too much physical memory. If you're running a Java app, you can use -hprof to look at what is taking up space in the heap. If you have exceeded virtual memory, you may need to increase the value of the the cluster-wide configuration variable `yarn.nodemanager.vmem-pmem-ratio`.
+
+### How do I include native libraries?
+
+Setting `-Djava.library.path` on the command line while launching a container can cause native libraries used by Hadoop to not be loaded correctly and can result in errors. It is cleaner to use `LD_LIBRARY_PATH` instead.
+
+Useful Links
+------------
+
+* [YARN Architecture](http://hadoop.apache.org/docs/current/hadoop-yarn/hadoop-yarn-site/YARN.html)
+
+* [YARN Capacity Scheduler](http://hadoop.apache.org/docs/current/hadoop-yarn/hadoop-yarn-site/CapacityScheduler.html)
+
+* [YARN Fair Scheduler](http://hadoop.apache.org/docs/current/hadoop-yarn/hadoop-yarn-site/FairScheduler.html)
+
+Sample Code
+-----------
+
+Yarn distributed shell: in `hadoop-yarn-applications-distributedshell` project after you set up your development environment.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/06aca7c6/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/YARN.md
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/YARN.md b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/YARN.md
new file mode 100644
index 0000000..f79272c
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/YARN.md
@@ -0,0 +1,42 @@
+<!---
+  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.
+-->
+
+Apache Hadoop NextGen MapReduce (YARN)
+==================
+
+MapReduce has undergone a complete overhaul in hadoop-0.23 and we now have, what we call, MapReduce 2.0 (MRv2) or YARN.
+
+The fundamental idea of MRv2 is to split up the two major functionalities of the JobTracker, resource management and job scheduling/monitoring, into separate daemons. The idea is to have a global ResourceManager (*RM*) and per-application ApplicationMaster (*AM*). An application is either a single job in the classical sense of Map-Reduce jobs or a DAG of jobs.
+
+The ResourceManager and per-node slave, the NodeManager (*NM*), form the data-computation framework. The ResourceManager is the ultimate authority that arbitrates resources among all the applications in the system.
+
+The per-application ApplicationMaster is, in effect, a framework specific library and is tasked with negotiating resources from the ResourceManager and working with the NodeManager(s) to execute and monitor the tasks.
+
+![MapReduce NextGen Architecture](./yarn_architecture.gif)
+
+The ResourceManager has two main components: Scheduler and ApplicationsManager.
+
+The Scheduler is responsible for allocating resources to the various running applications subject to familiar constraints of capacities, queues etc. The Scheduler is pure scheduler in the sense that it performs no monitoring or tracking of status for the application. Also, it offers no guarantees about restarting failed tasks either due to application failure or hardware failures. The Scheduler performs its scheduling function based the resource requirements of the applications; it does so based on the abstract notion of a resource *Container* which incorporates elements such as memory, cpu, disk, network etc. In the first version, only `memory` is supported.
+
+The Scheduler has a pluggable policy plug-in, which is responsible for partitioning the cluster resources among the various queues, applications etc. The current Map-Reduce schedulers such as the CapacityScheduler and the FairScheduler would be some examples of the plug-in.
+
+The CapacityScheduler supports `hierarchical queues` to allow for more predictable sharing of cluster resources
+
+The ApplicationsManager is responsible for accepting job-submissions, negotiating the first container for executing the application specific ApplicationMaster and provides the service for restarting the ApplicationMaster container on failure.
+
+The NodeManager is the per-machine framework agent who is responsible for containers, monitoring their resource usage (cpu, memory, disk, network) and reporting the same to the ResourceManager/Scheduler.
+
+The per-application ApplicationMaster has the responsibility of negotiating appropriate resource containers from the Scheduler, tracking their status and monitoring for progress.
+
+MRV2 maintains **API compatibility** with previous stable release (hadoop-1.x). This means that all Map-Reduce jobs should still run unchanged on top of MRv2 with just a recompile.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/06aca7c6/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/YarnCommands.md
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/YarnCommands.md b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/YarnCommands.md
new file mode 100644
index 0000000..28bb678
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/YarnCommands.md
@@ -0,0 +1,272 @@
+<!---
+  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.
+-->
+
+YARN Commands
+=============
+
+* [Overview](#Overview)
+* [User Commands](#User_Commands)
+    * [application](#application)
+    * [applicationattempt](#applicationattempt)
+    * [classpath](#classpath)
+    * [container](#container)
+    * [jar](#jar)
+    * [logs](#logs)
+    * [node](#node)
+    * [queue](#queue)
+    * [version](#version)
+* [Administration Commands](#Administration_Commands)
+    * [daemonlog](#daemonlog)
+    * [nodemanager](#nodemanager)
+    * [proxyserver](#proxyserver)
+    * [resourcemanager](#resourcemanager)
+    * [rmadmin](#rmadmin)
+    * [scmadmin](#scmadmin)
+    * [sharedcachemanager](#sharedcachemanager)
+    * [timelineserver](#timelineserver)
+* [Files](#Files)
+    * [etc/hadoop/hadoop-env.sh](#etchadoophadoop-env.sh)
+    * [etc/hadoop/yarn-env.sh](#etchadoopyarn-env.sh)
+    * [etc/hadoop/hadoop-user-functions.sh](#etchadoophadoop-user-functions.sh)
+    * [~/.hadooprc](#a.hadooprc)
+
+Overview
+--------
+
+YARN commands are invoked by the bin/yarn script. Running the yarn script without any arguments prints the description for all commands.
+
+Usage: `yarn [SHELL_OPTIONS] COMMAND [GENERIC_OPTIONS] [COMMAND_OPTIONS]`
+
+YARN has an option parsing framework that employs parsing generic options as well as running classes.
+
+| COMMAND\_OPTIONS | Description |
+|:---- |:---- |
+| SHELL\_OPTIONS | The common set of shell options. These are documented on the [Commands Manual](../../hadoop-project-dist/hadoop-common/CommandsManual.html#Shell_Options) page. |
+| GENERIC\_OPTIONS | The common set of options supported by multiple commands. See the Hadoop [Commands Manual](../../hadoop-project-dist/hadoop-common/CommandsManual.html#Generic_Options) for more information. |
+| COMMAND COMMAND\_OPTIONS | Various commands with their options are described in the following sections. The commands have been grouped into [User Commands](#User_Commands) and [Administration Commands](#Administration_Commands). |
+
+User Commands
+-------------
+
+Commands useful for users of a Hadoop cluster.
+
+### `application`
+
+Usage: `yarn application [options] `
+
+| COMMAND\_OPTIONS | Description |
+|:---- |:---- |
+| -appStates States | Works with -list to filter applications based on input comma-separated list of application states. The valid application state can be one of the following:  ALL, NEW, NEW\_SAVING, SUBMITTED, ACCEPTED, RUNNING, FINISHED, FAILED, KILLED |
+| -appTypes Types | Works with -list to filter applications based on input comma-separated list of application types. |
+| -list | Lists applications from the RM. Supports optional use of -appTypes to filter applications based on application type, and -appStates to filter applications based on application state. |
+| -kill ApplicationId | Kills the application. |
+| -status ApplicationId | Prints the status of the application. |
+
+Prints application(s) report/kill application
+
+### `applicationattempt`
+
+Usage: `yarn applicationattempt [options] `
+
+| COMMAND\_OPTIONS | Description |
+|:---- |:---- |
+| -help | Help |
+| -list ApplicationId | Lists applications attempts from the RM |
+| -status Application Attempt Id | Prints the status of the application attempt. |
+
+prints applicationattempt(s) report
+
+### `classpath`
+
+Usage: `yarn classpath`
+
+Prints the class path needed to get the Hadoop jar and the required libraries
+
+### `container`
+
+Usage: `yarn container [options] `
+
+| COMMAND\_OPTIONS | Description |
+|:---- |:---- |
+| -help | Help |
+| -list ApplicationId | Lists containers for the application attempt. |
+| -status ContainerId | Prints the status of the container. |
+
+prints container(s) report
+
+### `jar`
+
+Usage: `yarn jar <jar> [mainClass] args... `
+
+Runs a jar file. Users can bundle their YARN code in a jar file and execute it using this command.
+
+### `logs`
+
+Usage: `yarn logs -applicationId <application ID> [options] `
+
+| COMMAND\_OPTIONS | Description |
+|:---- |:---- |
+| -applicationId \<application ID\> | Specifies an application id |
+| -appOwner AppOwner | AppOwner (assumed to be current user if not specified) |
+| -containerId ContainerId | ContainerId (must be specified if node address is specified) |
+| -help | Help |
+| -nodeAddress NodeAddress | NodeAddress in the format nodename:port (must be specified if container id is specified) |
+
+Dump the container logs
+
+### `node`
+
+Usage: `yarn node [options] `
+
+| COMMAND\_OPTIONS | Description |
+|:---- |:---- |
+| -all | Works with -list to list all nodes. |
+| -list | Lists all running nodes. Supports optional use of -states to filter nodes based on node state, and -all to list all nodes. |
+| -states States | Works with -list to filter nodes based on input comma-separated list of node states. |
+| -status NodeId | Prints the status report of the node. |
+
+Prints node report(s)
+
+### `queue`
+
+Usage: `yarn queue [options] `
+
+| COMMAND\_OPTIONS | Description |
+|:---- |:---- |
+| -help | Help |
+| -status QueueName | Prints the status of the queue. |
+
+Prints queue information
+
+### `version`
+
+Usage: `yarn version`
+
+Prints the Hadoop version.
+
+Administration Commands
+-----------------------
+
+Commands useful for administrators of a Hadoop cluster.
+
+### `daemonlog`
+
+Usage:
+
+```
+   yarn daemonlog -getlevel <host:httpport> <classname> 
+   yarn daemonlog -setlevel <host:httpport> <classname> <level>
+```
+
+| COMMAND\_OPTIONS | Description |
+|:---- |:---- |
+| -getlevel `<host:httpport>` `<classname>` | Prints the log level of the log identified by a qualified `<classname>`, in the daemon running at `<host:httpport>`. This command internally connects to `http://<host:httpport>/logLevel?log=<classname>` |
+| -setlevel `<host:httpport> <classname> <level>` | Sets the log level of the log identified by a qualified `<classname>` in the daemon running at `<host:httpport>`. This command internally connects to `http://<host:httpport>/logLevel?log=<classname>&level=<level>` |
+
+Get/Set the log level for a Log identified by a qualified class name in the daemon.
+
+Example: `$ bin/yarn daemonlog -setlevel 127.0.0.1:8088 org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppImpl DEBUG`
+
+### `nodemanager`
+
+Usage: `yarn nodemanager`
+
+Start the NodeManager
+
+### `proxyserver`
+
+Usage: `yarn proxyserver`
+
+Start the web proxy server
+
+### `resourcemanager`
+
+Usage: `yarn resourcemanager [-format-state-store]`
+
+| COMMAND\_OPTIONS | Description |
+|:---- |:---- |
+| -format-state-store | Formats the RMStateStore. This will clear the RMStateStore and is useful if past applications are no longer needed. This should be run only when the ResourceManager is not running. |
+
+Start the ResourceManager
+
+### `rmadmin`
+
+Usage:
+
+```
+  yarn rmadmin [-refreshQueues]
+               [-refreshNodes]
+               [-refreshUserToGroupsMapping] 
+               [-refreshSuperUserGroupsConfiguration]
+               [-refreshAdminAcls] 
+               [-refreshServiceAcl]
+               [-getGroups [username]]
+               [-transitionToActive [--forceactive] [--forcemanual] <serviceId>]
+               [-transitionToStandby [--forcemanual] <serviceId>]
+               [-failover [--forcefence] [--forceactive] <serviceId1> <serviceId2>]
+               [-getServiceState <serviceId>]
+               [-checkHealth <serviceId>]
+               [-help [cmd]]
+```
+
+| COMMAND\_OPTIONS | Description |
+|:---- |:---- |
+| -refreshQueues | Reload the queues' acls, states and scheduler specific properties. ResourceManager will reload the mapred-queues configuration file. |
+| -refreshNodes | Refresh the hosts information at the ResourceManager. |
+| -refreshUserToGroupsMappings | Refresh user-to-groups mappings. |
+| -refreshSuperUserGroupsConfiguration | Refresh superuser proxy groups mappings. |
+| -refreshAdminAcls | Refresh acls for administration of ResourceManager |
+| -refreshServiceAcl | Reload the service-level authorization policy file ResourceManager will reload the authorization policy file. |
+| -getGroups [username] | Get groups the specified user belongs to. |
+| -transitionToActive [--forceactive] [--forcemanual] \<serviceId\> | Transitions the service into Active state. Try to make the target active without checking that there is no active node if the --forceactive option is used. This command can not be used if automatic failover is enabled. Though you can override this by --forcemanual option, you need caution. |
+| -transitionToStandby [--forcemanual] \<serviceId\> | Transitions the service into Standby state. This command can not be used if automatic failover is enabled. Though you can override this by --forcemanual option, you need caution. |
+| -failover [--forceactive] \<serviceId1\> \<serviceId2\> | Initiate a failover from serviceId1 to serviceId2. Try to failover to the target service even if it is not ready if the --forceactive option is used. This command can not be used if automatic failover is enabled. |
+| -getServiceState \<serviceId\> | Returns the state of the service. |
+| -checkHealth \<serviceId\> | Requests that the service perform a health check. The RMAdmin tool will exit with a non-zero exit code if the check fails. |
+| -help [cmd] | Displays help for the given command or all commands if none is specified. |
+
+Runs ResourceManager admin client
+
+### scmadmin
+
+Usage: `yarn scmadmin [options] `
+
+| COMMAND\_OPTIONS | Description |
+|:---- |:---- |
+| -help | Help |
+| -runCleanerTask | Runs the cleaner task |
+
+Runs Shared Cache Manager admin client
+
+### sharedcachemanager
+
+Usage: `yarn sharedcachemanager`
+
+Start the Shared Cache Manager
+
+### timelineserver
+
+Usage: `yarn timelineserver`
+
+Start the TimeLineServer
+
+Files
+-----
+
+| File | Description |
+|:---- |:---- |
+| etc/hadoop/hadoop-env.sh | This file stores the global settings used by all Hadoop shell commands. |
+| etc/hadoop/yarn-env.sh | This file stores overrides used by all YARN shell commands. |
+| etc/hadoop/hadoop-user-functions.sh | This file allows for advanced users to override some shell functionality. |
+| ~/.hadooprc | This stores the personal environment for an individual user. It is processed after the `hadoop-env.sh`, `hadoop-user-functions.sh`, and `yarn-env.sh` files and can contain the same settings. |

http://git-wip-us.apache.org/repos/asf/hadoop/blob/06aca7c6/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/index.md
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/index.md b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/index.md
new file mode 100644
index 0000000..9637ea0
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/index.md
@@ -0,0 +1,75 @@
+<!---
+  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.
+-->
+
+MapReduce NextGen aka YARN aka MRv2
+===================================
+
+The new architecture introduced in hadoop-0.23, divides the two major functions of the JobTracker: resource management and job life-cycle management into separate components.
+
+The new ResourceManager manages the global assignment of compute resources to applications and the per-application ApplicationMaster manages the application’s scheduling and coordination.
+
+An application is either a single job in the sense of classic MapReduce jobs or a DAG of such jobs.
+
+The ResourceManager and per-machine NodeManager daemon, which manages the user processes on that machine, form the computation fabric.
+
+The per-application ApplicationMaster is, in effect, a framework specific library and is tasked with negotiating resources from the ResourceManager and working with the NodeManager(s) to execute and monitor the tasks.
+
+More details are available in the [Architecture](./YARN.html) document.
+
+Documentation Index
+===================
+
+YARN
+----
+
+* [YARN Architecture](./YARN.html)
+
+* [Capacity Scheduler](./CapacityScheduler.html)
+
+* [Fair Scheduler](./FairScheduler.html)
+
+* [ResourceManager Restart](./ResourceManagerRestart.htaml)
+
+* [ResourceManager HA](./ResourceManagerHA.html)
+
+* [Web Application Proxy](./WebApplicationProxy.html)
+
+* [YARN Timeline Server](./TimelineServer.html)
+
+* [Writing YARN Applications](./WritingYarnApplications.html)
+
+* [YARN Commands](./YarnCommands.html)
+
+* [Scheduler Load Simulator](#hadoop-slsSchedulerLoadSimulator.html)
+
+* [NodeManager Restart](./NodeManagerRestart.html)
+
+* [DockerContainerExecutor](./DockerContainerExecutor.html)
+
+* [Using CGroups](./NodeManagerCGroups.html)
+
+* [Secure Containers](./SecureContainer.html)
+
+* [Registry](./registry/index.html)
+
+YARN REST APIs
+--------------
+
+* [Introduction](./WebServicesIntro.html)
+
+* [Resource Manager](./ResourceManagerRest.html)
+
+* [Node Manager](./NodeManagerRest.html)
+
+


[22/50] [abbrv] hadoop git commit: Revert "HDFS-7769. TestHDFSCLI should not create files in hdfs project root dir."

Posted by zh...@apache.org.
Revert "HDFS-7769. TestHDFSCLI should not create files in hdfs project root dir."

This reverts commit 7c6b6547eeed110e1a842e503bfd33afe04fa814.

Conflicts:
	hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt


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

Branch: refs/heads/HDFS-7285
Commit: 04f0dae7c982c3af7e96811a92a1cb6aeda74f9f
Parents: 645ebb9
Author: Tsz-Wo Nicholas Sze <sz...@hortonworks.com>
Authored: Fri Feb 27 18:25:32 2015 +0800
Committer: Zhe Zhang <zh...@cloudera.com>
Committed: Mon Mar 2 09:13:53 2015 -0800

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt                      | 3 ---
 .../hadoop-hdfs/src/test/resources/testHDFSConf.xml              | 4 ++--
 2 files changed, 2 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/04f0dae7/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 ae83898..ba553dc 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -975,9 +975,6 @@ Release 2.7.0 - UNRELEASED
     HDFS-7714. Simultaneous restart of HA NameNodes and DataNode can cause
     DataNode to register successfully with only one NameNode.(vinayakumarb)
 
-    HDFS-7769. TestHDFSCLI should not create files in hdfs project root dir.
-    (szetszwo)
-
     HDFS-7753. Fix Multithreaded correctness Warnings in BackupImage.
     (Rakesh R and shv)
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/04f0dae7/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testHDFSConf.xml
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testHDFSConf.xml b/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testHDFSConf.xml
index 2d3de1f..e59b05a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testHDFSConf.xml
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testHDFSConf.xml
@@ -16483,8 +16483,8 @@
         <command>-fs NAMENODE -mkdir -p /user/USERNAME/dir1</command>
         <command>-fs NAMENODE -copyFromLocal CLITEST_DATA/data15bytes /user/USERNAME/dir1</command>
         <command>-fs NAMENODE -copyFromLocal CLITEST_DATA/data30bytes /user/USERNAME/dir1</command>
-        <command>-fs NAMENODE -getmerge /user/USERNAME/dir1 CLITEST_DATA/file</command>
-        <command>-cat CLITEST_DATA/file</command>
+        <command>-fs NAMENODE -getmerge /user/USERNAME/dir1 data</command>
+        <command>-cat data</command>
       </test-commands>
       <cleanup-commands>
         <command>-fs NAMENODE -rm -r /user/USERNAME</command>


[34/50] [abbrv] hadoop git commit: YARN-3168. Convert site documentation from apt to markdown (Gururaj Shetty via aw)

Posted by zh...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/06aca7c6/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/apt/ResourceManagerRestart.apt.vm
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/apt/ResourceManagerRestart.apt.vm b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/apt/ResourceManagerRestart.apt.vm
deleted file mode 100644
index a08c19d..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/apt/ResourceManagerRestart.apt.vm
+++ /dev/null
@@ -1,298 +0,0 @@
-~~ 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.
-
-  ---
-  ResourceManager Restart
-  ---
-  ---
-  ${maven.build.timestamp}
-
-ResourceManager Restart
-
-%{toc|section=1|fromDepth=0}
-
-* {Overview}
-
-  ResourceManager is the central authority that manages resources and schedules
-  applications running atop of YARN. Hence, it is potentially a single point of
-  failure in a Apache YARN cluster.
-
-  This document gives an overview of ResourceManager Restart, a feature that
-  enhances ResourceManager to keep functioning across restarts and also makes
-  ResourceManager down-time invisible to end-users.
-
-  ResourceManager Restart feature is divided into two phases:
-
-  ResourceManager Restart Phase 1 (Non-work-preserving RM restart):
-  Enhance RM to persist application/attempt state
-  and other credentials information in a pluggable state-store. RM will reload
-  this information from state-store upon restart and re-kick the previously
-  running applications. Users are not required to re-submit the applications.
-
-  ResourceManager Restart Phase 2 (Work-preserving RM restart):
-  Focus on re-constructing the running state of ResourceManager by combining
-  the container statuses from NodeManagers and container requests from ApplicationMasters
-  upon restart. The key difference from phase 1 is that previously running applications
-  will not be killed after RM restarts, and so applications won't lose its work
-  because of RM outage.
-
-* {Feature}
-
-** Phase 1: Non-work-preserving RM restart
-
-  As of Hadoop 2.4.0 release, only ResourceManager Restart Phase 1 is implemented which
-  is described below.
-
-  The overall concept is that RM will persist the application metadata
-  (i.e. ApplicationSubmissionContext) in
-  a pluggable state-store when client submits an application and also saves the final status
-  of the application such as the completion state (failed, killed, finished) 
-  and diagnostics when the application completes. Besides, RM also saves
-  the credentials like security keys, tokens to work in a secure environment.
-  Any time RM shuts down, as long as the required information (i.e.application metadata
-  and the alongside credentials if running in a secure environment) is available
-  in the state-store, when RM restarts, it can pick up the application metadata
-  from the state-store and re-submit the application. RM won't re-submit the
-  applications if they were already completed (i.e. failed, killed, finished)
-  before RM went down.
-
-  NodeManagers and clients during the down-time of RM will keep polling RM until 
-  RM comes up. When RM becomes alive, it will send a re-sync command to
-  all the NodeManagers and ApplicationMasters it was talking to via heartbeats.
-  As of Hadoop 2.4.0 release, the behaviors for NodeManagers and ApplicationMasters to handle this command
-  are: NMs will kill all its managed containers and re-register with RM. From the
-  RM's perspective, these re-registered NodeManagers are similar to the newly joining NMs. 
-  AMs(e.g. MapReduce AM) are expected to shutdown when they receive the re-sync command.
-  After RM restarts and loads all the application metadata, credentials from state-store
-  and populates them into memory, it will create a new
-  attempt (i.e. ApplicationMaster) for each application that was not yet completed
-  and re-kick that application as usual. As described before, the previously running
-  applications' work is lost in this manner since they are essentially killed by
-  RM via the re-sync command on restart.
-
-** Phase 2: Work-preserving RM restart
-
-  As of Hadoop 2.6.0, we further enhanced RM restart feature to address the problem 
-  to not kill any applications running on YARN cluster if RM restarts.
-
-  Beyond all the groundwork that has been done in Phase 1 to ensure the persistency
-  of application state and reload that state on recovery, Phase 2 primarily focuses
-  on re-constructing the entire running state of YARN cluster, the majority of which is
-  the state of the central scheduler inside RM which keeps track of all containers' life-cycle,
-  applications' headroom and resource requests, queues' resource usage etc. In this way,
-  RM doesn't need to kill the AM and re-run the application from scratch as it is
-  done in Phase 1. Applications can simply re-sync back with RM and
-  resume from where it were left off.
-
-  RM recovers its runing state by taking advantage of the container statuses sent from all NMs.
-  NM will not kill the containers when it re-syncs with the restarted RM. It continues
-  managing the containers and send the container statuses across to RM when it re-registers.
-  RM reconstructs the container instances and the associated applications' scheduling status by
-  absorbing these containers' information. In the meantime, AM needs to re-send the
-  outstanding resource requests to RM because RM may lose the unfulfilled requests when it shuts down.
-  Application writers using AMRMClient library to communicate with RM do not need to
-  worry about the part of AM re-sending resource requests to RM on re-sync, as it is
-  automatically taken care by the library itself.
-
-* {Configurations}
-
-** Enable RM Restart.
-
-*--------------------------------------+--------------------------------------+
-|| Property                            || Value                                |
-*--------------------------------------+--------------------------------------+
-| <<<yarn.resourcemanager.recovery.enabled>>> | |
-| | <<<true>>> |
-*--------------------------------------+--------------------------------------+ 
-
-
-** Configure the state-store for persisting the RM state.
-
-
-*--------------------------------------*--------------------------------------+
-|| Property                            || Description                        |
-*--------------------------------------+--------------------------------------+
-| <<<yarn.resourcemanager.store.class>>> | |
-| | The class name of the state-store to be used for saving application/attempt |
-| | state and the credentials. The available state-store implementations are  |
-| | <<<org.apache.hadoop.yarn.server.resourcemanager.recovery.ZKRMStateStore>>> |
-| | , a ZooKeeper based state-store implementation and  |
-| | <<<org.apache.hadoop.yarn.server.resourcemanager.recovery.FileSystemRMStateStore>>> |
-| | , a Hadoop FileSystem based state-store implementation like HDFS and local FS. |
-| | <<<org.apache.hadoop.yarn.server.resourcemanager.recovery.LeveldbRMStateStore>>>, |
-| | a LevelDB based state-store implementation. |
-| | The default value is set to |
-| | <<<org.apache.hadoop.yarn.server.resourcemanager.recovery.FileSystemRMStateStore>>>. |
-*--------------------------------------+--------------------------------------+ 
-
-** How to choose the state-store implementation.
-
-    <<ZooKeeper based state-store>>: User is free to pick up any storage to set up RM restart,
-    but must use ZooKeeper based state-store to support RM HA. The reason is that only ZooKeeper
-    based state-store supports fencing mechanism to avoid a split-brain situation where multiple
-    RMs assume they are active and can edit the state-store at the same time.
-
-    <<FileSystem based state-store>>: HDFS and local FS based state-store are supported. 
-    Fencing mechanism is not supported.
-
-    <<LevelDB based state-store>>: LevelDB based state-store is considered more light weight than HDFS and ZooKeeper
-    based state-store. LevelDB supports better atomic operations, fewer I/O ops per state update,
-    and far fewer total files on the filesystem. Fencing mechanism is not supported.
-
-** Configurations for Hadoop FileSystem based state-store implementation.
-
-    Support both HDFS and local FS based state-store implementation. The type of file system to
-    be used is determined by the scheme of URI. e.g. <<<hdfs://localhost:9000/rmstore>>> uses HDFS as the storage and
-    <<<file:///tmp/yarn/rmstore>>> uses local FS as the storage. If no
-    scheme(<<<hdfs://>>> or <<<file://>>>) is specified in the URI, the type of storage to be used is
-    determined by <<<fs.defaultFS>>> defined in <<<core-site.xml>>>.
-
-    Configure the URI where the RM state will be saved in the Hadoop FileSystem state-store.
-
-*--------------------------------------+--------------------------------------+
-|| Property                            || Description                        |
-*--------------------------------------+--------------------------------------+
-| <<<yarn.resourcemanager.fs.state-store.uri>>> | |
-| | URI pointing to the location of the FileSystem path where RM state will |
-| | be stored (e.g. hdfs://localhost:9000/rmstore). |
-| | Default value is <<<${hadoop.tmp.dir}/yarn/system/rmstore>>>. |
-| | If FileSystem name is not provided, <<<fs.default.name>>> specified in |
-| | <<conf/core-site.xml>> will be used. |
-*--------------------------------------+--------------------------------------+ 
-
-    Configure the retry policy state-store client uses to connect with the Hadoop
-    FileSystem.
-
-*--------------------------------------+--------------------------------------+
-|| Property                            || Description                        |
-*--------------------------------------+--------------------------------------+
-| <<<yarn.resourcemanager.fs.state-store.retry-policy-spec>>> | |
-| | Hadoop FileSystem client retry policy specification. Hadoop FileSystem client retry | 
-| | is always enabled. Specified in pairs of sleep-time and number-of-retries | 
-| | i.e. (t0, n0), (t1, n1), ..., the first n0 retries sleep t0 milliseconds on |
-| | average, the following n1 retries sleep t1 milliseconds on average, and so on. |
-| | Default value is (2000, 500) |
-*--------------------------------------+--------------------------------------+ 
-
-** Configurations for ZooKeeper based state-store implementation.
-  
-    Configure the ZooKeeper server address and the root path where the RM state is stored.
-
-*--------------------------------------+--------------------------------------+
-|| Property                            || Description                        |
-*--------------------------------------+--------------------------------------+
-| <<<yarn.resourcemanager.zk-address>>> | |
-| | Comma separated list of Host:Port pairs. Each corresponds to a ZooKeeper server |
-| | (e.g. "127.0.0.1:3000,127.0.0.1:3001,127.0.0.1:3002") to be used by the RM |
-| | for storing RM state. |
-*--------------------------------------+--------------------------------------+
-| <<<yarn.resourcemanager.zk-state-store.parent-path>>> | |
-| | The full path of the root znode where RM state will be stored. |
-| | Default value is /rmstore. |
-*--------------------------------------+--------------------------------------+
-
-    Configure the retry policy state-store client uses to connect with the ZooKeeper server.
-
-*--------------------------------------+--------------------------------------+
-|| Property                            || Description                        |
-*--------------------------------------+--------------------------------------+
-| <<<yarn.resourcemanager.zk-num-retries>>> | |
-| | Number of times RM tries to connect to ZooKeeper server if the connection is lost. |
-| | Default value is 500. |
-*--------------------------------------+--------------------------------------+
-| <<<yarn.resourcemanager.zk-retry-interval-ms>>> |
-| | The interval in milliseconds between retries when connecting to a ZooKeeper server. |
-| | Default value is 2 seconds. |
-*--------------------------------------+--------------------------------------+
-| <<<yarn.resourcemanager.zk-timeout-ms>>> | |
-| | ZooKeeper session timeout in milliseconds. This configuration is used by  |
-| | the ZooKeeper server to determine when the session expires. Session expiration |
-| | happens when the server does not hear from the client (i.e. no heartbeat) within the session |
-| | timeout period specified by this configuration. Default |
-| | value is 10 seconds |
-*--------------------------------------+--------------------------------------+
-
-    Configure the ACLs to be used for setting permissions on ZooKeeper znodes.
-
-*--------------------------------------+--------------------------------------+
-|| Property                            || Description                        |
-*--------------------------------------+--------------------------------------+
-| <<<yarn.resourcemanager.zk-acl>>> | |
-| | ACLs to be used for setting permissions on ZooKeeper znodes. Default value is <<<world:anyone:rwcda>>> |
-*--------------------------------------+--------------------------------------+
-
-** Configurations for LevelDB based state-store implementation.
-
-*--------------------------------------+--------------------------------------+
-|| Property                            || Description                        |
-*--------------------------------------+--------------------------------------+
-| <<<yarn.resourcemanager.leveldb-state-store.path>>> | |
-| | Local path where the RM state will be stored. |
-| | Default value is <<<${hadoop.tmp.dir}/yarn/system/rmstore>>> |
-*--------------------------------------+--------------------------------------+
-
-
-**  Configurations for work-preserving RM recovery.
-
-*--------------------------------------+--------------------------------------+
-|| Property                            || Description                        |
-*--------------------------------------+--------------------------------------+
-| <<<yarn.resourcemanager.work-preserving-recovery.scheduling-wait-ms>>> | |
-| | Set the amount of time RM waits before allocating new |
-| | containers on RM work-preserving recovery. Such wait period gives RM a chance | 
-| | to settle down resyncing with NMs in the cluster on recovery, before assigning|
-| |  new containers to applications.|
-*--------------------------------------+--------------------------------------+
-
-* {Notes}
-
-  ContainerId string format is changed if RM restarts with work-preserving recovery enabled.
-  It used to be such format:
-
-   Container_\{clusterTimestamp\}_\{appId\}_\{attemptId\}_\{containerId\}, e.g. Container_1410901177871_0001_01_000005.
-
-  It is now changed to:
-
-   Container_<<e\{epoch\}>>_\{clusterTimestamp\}_\{appId\}_\{attemptId\}_\{containerId\}, e.g. Container_<<e17>>_1410901177871_0001_01_000005.
- 
-  Here, the additional epoch number is a
-  monotonically increasing integer which starts from 0 and is increased by 1 each time
-  RM restarts. If epoch number is 0, it is omitted and the containerId string format
-  stays the same as before.
-
-* {Sample configurations}
-
-   Below is a minimum set of configurations for enabling RM work-preserving restart using ZooKeeper based state store.
-
-+---+
-  <property>
-    <description>Enable RM to recover state after starting. If true, then 
-    yarn.resourcemanager.store.class must be specified</description>
-    <name>yarn.resourcemanager.recovery.enabled</name>
-    <value>true</value>
-  </property>
-
-  <property>
-    <description>The class to use as the persistent store.</description>
-    <name>yarn.resourcemanager.store.class</name>
-    <value>org.apache.hadoop.yarn.server.resourcemanager.recovery.ZKRMStateStore</value>
-  </property>
-
-  <property>
-    <description>Comma separated list of Host:Port pairs. Each corresponds to a ZooKeeper server
-    (e.g. "127.0.0.1:3000,127.0.0.1:3001,127.0.0.1:3002") to be used by the RM for storing RM state.
-    This must be supplied when using org.apache.hadoop.yarn.server.resourcemanager.recovery.ZKRMStateStore
-    as the value for yarn.resourcemanager.store.class</description>
-    <name>yarn.resourcemanager.zk-address</name>
-    <value>127.0.0.1:2181</value>
-  </property>
-+---+
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/06aca7c6/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/apt/SecureContainer.apt.vm
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/apt/SecureContainer.apt.vm b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/apt/SecureContainer.apt.vm
deleted file mode 100644
index 0365bf7..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/apt/SecureContainer.apt.vm
+++ /dev/null
@@ -1,176 +0,0 @@
-~~ 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.
-
-  ---
-  YARN Secure Containers
-  ---
-  ---
-  ${maven.build.timestamp}
-
-YARN Secure Containers
-
-%{toc|section=1|fromDepth=0|toDepth=3}
-
-* {Overview}
-
-  YARN containers in a secure cluster use the operating system facilities to offer
-  execution isolation for containers. Secure containers execute under the credentials
-  of the job user. The operating system enforces access restriction for the container.
-  The container must run as the use that submitted the application.
-  
-  Secure Containers work only in the context of secured YARN clusters.
-  
-  ** Container isolation requirements
-  
-    The container executor must access the local files and directories needed by the 
-    container such as jars, configuration files, log files, shared objects etc. Although
-    it is launched by the NodeManager, the container should not have access to the 
-    NodeManager private files and configuration. Container running applications 
-    submitted by different users should be isolated and unable to access each other
-    files and directories. Similar requirements apply to other system non-file securable 
-    objects like named pipes, critical sections, LPC queues, shared memory etc.
-    
-    
-  ** Linux Secure Container Executor
-
-    On Linux environment the secure container executor is the <<<LinuxContainerExecutor>>>.
-    It uses an external program called the <<container-executor>>> to launch the container.
-    This program has the <<<setuid>>> access right flag set which allows it to launch 
-    the container with the permissions of the YARN application user.
-    
-  *** Configuration
-
-      The configured directories for <<<yarn.nodemanager.local-dirs>>> and 
-      <<<yarn.nodemanager.log-dirs>>> must be owned by the configured NodeManager user
-      (<<<yarn>>>) and group (<<<hadoop>>>). The permission set on these directories must
-      be <<<drwxr-xr-x>>>.
-      
-      The <<<container-executor>>> program must be owned by <<<root>>> and have the
-      permission set <<<---sr-s--->>>.
-
-      To configure the <<<NodeManager>>> to use the <<<LinuxContainerExecutor>>> set the following 
-      in the <<conf/yarn-site.xml>>:
-
-+---+
-<property>
-  <name>yarn.nodemanager.container-executor.class</name>
-  <value>org.apache.hadoop.yarn.server.nodemanager.LinuxContainerExecutor</value>
-</property>
-
-<property>
-  <name>yarn.nodemanager.linux-container-executor.group</name>
-  <value>hadoop</value>
-</property>
-+---+
-
-      Additionally the LCE requires the <<<container-executor.cfg>>> file, which is read by the
-      <<<container-executor>>> program. 
-
-+---+
-yarn.nodemanager.linux-container-executor.group=#configured value of yarn.nodemanager.linux-container-executor.group
-banned.users=#comma separated list of users who can not run applications
-allowed.system.users=#comma separated list of allowed system users
-min.user.id=1000#Prevent other super-users
-+---+
-
-   
-  ** Windows Secure Container Executor (WSCE)
-  
-    The Windows environment secure container executor is the <<<WindowsSecureContainerExecutor>>>.
-    It uses the Windows S4U infrastructure to launch the container as the 
-    YARN application user. The WSCE requires the presense of the <<<hadoopwinutilsvc>>>  service. This services
-    is hosted by <<<%HADOOP_HOME%\bin\winutils.exe>>> started with the <<<service>>> command line argument. This
-    service offers some privileged operations that require LocalSystem authority so that the NM is not required
-    to run the entire JVM and all the NM code in an elevated context. The NM interacts with the <<<hadoopwintulsvc>>>
-    service by means of Local RPC (LRPC) via calls JNI to the RCP client hosted in <<<hadoop.dll>>>.
-    
-  *** Configuration
-  
-      To configure the <<<NodeManager>>> to use the <<<WindowsSecureContainerExecutor>>> 
-      set the following in the <<conf/yarn-site.xml>>:
-
-+---+
-<property>
-  <name>yarn.nodemanager.container-executor.class</name>
-  <value>org.apache.hadoop.yarn.server.nodemanager.WindowsSecureContainerExecutor</value>
-</property>
-
-<property>
-  <name>yarn.nodemanager.windows-secure-container-executor.group</name>
-  <value>yarn</value>
-</property>
-+---+
-  *** wsce-site.xml
-  
-      The hadoopwinutilsvc uses <<<%HADOOP_HOME%\etc\hadoop\wsce_site.xml>>> to configure access to the privileged operations.
-
-+---+
-  <property>
-    <name>yarn.nodemanager.windows-secure-container-executor.impersonate.allowed</name>
-    <value>HadoopUsers</value>
-  </property>
-  
-  <property>
-    <name>yarn.nodemanager.windows-secure-container-executor.impersonate.denied</name>
-    <value>HadoopServices,Administrators</value>
-  </property>
-  
-  <property>
-    <name>yarn.nodemanager.windows-secure-container-executor.allowed</name>
-    <value>nodemanager</value>
-  </property>
-
-  <property>
-    <name>yarn.nodemanager.windows-secure-container-executor.local-dirs</name>
-    <value>nm-local-dir, nm-log-dirs</value>
-  </property>
-
-  <property>
-    <name>yarn.nodemanager.windows-secure-container-executor.job-name</name>
-    <value>nodemanager-job-name</value>
-  </property>  
-+---+
-      
-      <<<yarn.nodemanager.windows-secure-container-executor.allowed>>> should contain the name of the service account running the 
-      nodemanager. This user will be allowed to access the hadoopwintuilsvc functions.
-      
-      <<<yarn.nodemanager.windows-secure-container-executor.impersonate.allowed>>> should contain the users that are allowed to create
-      containers in the cluster. These users will be allowed to be impersonated by hadoopwinutilsvc.
-      
-      <<<yarn.nodemanager.windows-secure-container-executor.impersonate.denied>>> should contain users that are explictly forbiden from
-      creating containers. hadoopwinutilsvc will refuse to impersonate these users.
-
-      <<<yarn.nodemanager.windows-secure-container-executor.local-dirs>>> should contain the nodemanager local dirs. hadoopwinutilsvc will
-      allow only file operations under these directories. This should contain the same values as <<<${yarn.nodemanager.local-dirs}, ${yarn.nodemanager.log-dirs}>>> 
-      but note that hadoopwinutilsvc XML configuration processing does not do substitutions so the value must be the final value. All paths 
-      must be absolute and no environment variable substitution will be performed. The paths are compared LOCAL_INVARIANT case insensitive string comparison,
-      the file path validated must start with one of the paths listed in local-dirs configuration. Use comma as path separator:<<<,>>>
-
-      <<<yarn.nodemanager.windows-secure-container-executor.job-name>>> should contain an Windows NT job name that all containers should be added to. 
-      This configuration is optional. If not set, the container is not added to a global NodeManager job. Normally this should be set to the job that the NM is assigned to, 
-      so that killing the NM kills also all containers. Hadoopwinutilsvc will not attempt to create this job, the job must exists when the container is launched.
-      If the value is set and the job does not exists, container launch will fail with error 2 <<<The system cannot find the file specified>>>.
-      Note that this global NM job is not related to the container job, which always gets created for each container and is named after the container ID.
-      This setting controls a global job that spans all containers and the parent NM, and as such it requires nested jobs. 
-      Nested jobs are available only post Windows 8 and Windows Server 2012.
-      
-  *** Useful Links
-    
-    * {{{http://msdn.microsoft.com/en-us/magazine/cc188757.aspx}Exploring S4U Kerberos Extensions in Windows Server 2003}}
-    
-    * {{{http://msdn.microsoft.com/en-us/library/windows/desktop/hh448388(v=vs.85).aspx}Nested Jobs}}
-
-    * {{{https://issues.apache.org/jira/browse/YARN-1063}Winutils needs ability to create task as domain user}}
-    
-    * {{{https://issues.apache.org/jira/browse/YARN-1972}Implement secure Windows Container Executor}}
-
-    * {{{https://issues.apache.org/jira/browse/YARN-2198}Remove the need to run NodeManager as privileged account for Windows Secure Container Executor}}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/06aca7c6/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/apt/TimelineServer.apt.vm
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/apt/TimelineServer.apt.vm b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/apt/TimelineServer.apt.vm
deleted file mode 100644
index 7bb504e..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/apt/TimelineServer.apt.vm
+++ /dev/null
@@ -1,260 +0,0 @@
-~~ 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.
-
-  ---
-  YARN Timeline Server
-  ---
-  ---
-  ${maven.build.timestamp}
-
-YARN Timeline Server
-
-%{toc|section=1|fromDepth=0|toDepth=3}
-
-* Overview
-
-  Storage and retrieval of applications' current as well as historic
-  information in a generic fashion is solved in YARN through the Timeline
-  Server (previously also called Generic Application History Server). This
-  serves two responsibilities:
-
-  ** Generic information about completed applications
-  
-    Generic information includes application level data like queue-name, user
-    information etc in the ApplicationSubmissionContext, list of
-    application-attempts that ran for an application, information about each
-    application-attempt, list of containers run under each application-attempt,
-    and information about each container. Generic data is stored by
-    ResourceManager to a history-store (default implementation on a file-system)
-    and used by the web-UI to display information about completed applications.
-
-  ** Per-framework information of running and completed applications
-
-    Per-framework information is completely specific to an application or
-    framework. For example, Hadoop MapReduce framework can include pieces of
-    information like number of map tasks, reduce tasks, counters etc.
-    Application developers can publish the specific information to the Timeline
-    server via TimelineClient from within a client, the ApplicationMaster
-    and/or the application's containers. This information is then queryable via
-    REST APIs for rendering by application/framework specific UIs. 
-
-* Current Status
-
-  Timeline sever is a work in progress. The basic storage and retrieval of
-  information, both generic and framework specific, are in place. Timeline
-  server doesn't work in secure mode yet. The generic information and the
-  per-framework information are today collected and presented separately and
-  thus are not integrated well together. Finally, the per-framework information
-  is only available via RESTful APIs, using JSON type content - ability to
-  install framework specific UIs in YARN isn't supported yet.
-
-* Basic Configuration
-
-  Users need to configure the Timeline server before starting it. The simplest
-  configuration you should add in <<<yarn-site.xml>>> is to set the hostname of
-  the Timeline server:
-
-+---+
-<property>
-  <description>The hostname of the Timeline service web application.</description>
-  <name>yarn.timeline-service.hostname</name>
-  <value>0.0.0.0</value>
-</property>
-+---+
-
-* Advanced Configuration
-
-  In addition to the hostname, admins can also configure whether the service is
-  enabled or not, the ports of the RPC and the web interfaces, and the number
-  of RPC handler threads.
-
-+---+
-
-<property>
-  <description>Address for the Timeline server to start the RPC server.</description>
-  <name>yarn.timeline-service.address</name>
-  <value>${yarn.timeline-service.hostname}:10200</value>
-</property>
-
-<property>
-  <description>The http address of the Timeline service web application.</description>
-  <name>yarn.timeline-service.webapp.address</name>
-  <value>${yarn.timeline-service.hostname}:8188</value>
-</property>
-
-<property>
-  <description>The https address of the Timeline service web application.</description>
-  <name>yarn.timeline-service.webapp.https.address</name>
-  <value>${yarn.timeline-service.hostname}:8190</value>
-</property>
-
-<property>
-  <description>Handler thread count to serve the client RPC requests.</description>
-  <name>yarn.timeline-service.handler-thread-count</name>
-  <value>10</value>
-</property>
-
-<property>
-  <description>Enables cross-origin support (CORS) for web services where
-  cross-origin web response headers are needed. For example, javascript making
-  a web services request to the timeline server.</description>
-  <name>yarn.timeline-service.http-cross-origin.enabled</name>
-  <value>false</value>
-</property>
-
-<property>
-  <description>Comma separated list of origins that are allowed for web
-  services needing cross-origin (CORS) support. Wildcards (*) and patterns
-  allowed</description>
-  <name>yarn.timeline-service.http-cross-origin.allowed-origins</name>
-  <value>*</value>
-</property>
-
-<property>
-  <description>Comma separated list of methods that are allowed for web
-  services needing cross-origin (CORS) support.</description>
-  <name>yarn.timeline-service.http-cross-origin.allowed-methods</name>
-  <value>GET,POST,HEAD</value>
-</property>
-
-<property>
-  <description>Comma separated list of headers that are allowed for web
-  services needing cross-origin (CORS) support.</description>
-  <name>yarn.timeline-service.http-cross-origin.allowed-headers</name>
-  <value>X-Requested-With,Content-Type,Accept,Origin</value>
-</property>
-
-<property>
-  <description>The number of seconds a pre-flighted request can be cached
-  for web services needing cross-origin (CORS) support.</description>
-  <name>yarn.timeline-service.http-cross-origin.max-age</name>
-  <value>1800</value>
-</property>
-+---+
-
-* Generic-data related Configuration
-
-  Users can specify whether the generic data collection is enabled or not, and
-  also choose the storage-implementation class for the generic data. There are
-  more configurations related to generic data collection, and users can refer
-  to <<<yarn-default.xml>>> for all of them.
-
-+---+
-<property>
-  <description>Indicate to ResourceManager as well as clients whether
-  history-service is enabled or not. If enabled, ResourceManager starts
-  recording historical data that Timelien service can consume. Similarly,
-  clients can redirect to the history service when applications
-  finish if this is enabled.</description>
-  <name>yarn.timeline-service.generic-application-history.enabled</name>
-  <value>false</value>
-</property>
-
-<property>
-  <description>Store class name for history store, defaulting to file system
-  store</description>
-  <name>yarn.timeline-service.generic-application-history.store-class</name>
-  <value>org.apache.hadoop.yarn.server.applicationhistoryservice.FileSystemApplicationHistoryStore</value>
-</property>
-+---+
-
-* Per-framework-date related Configuration
-
-  Users can specify whether per-framework data service is enabled or not,
-  choose the store implementation for the per-framework data, and tune the
-  retention of the per-framework data. There are more configurations related to
-  per-framework data service, and users can refer to <<<yarn-default.xml>>> for
-  all of them.
-
-+---+
-<property>
-  <description>Indicate to clients whether Timeline service is enabled or not.
-  If enabled, the TimelineClient library used by end-users will post entities
-  and events to the Timeline server.</description>
-  <name>yarn.timeline-service.enabled</name>
-  <value>true</value>
-</property>
-
-<property>
-  <description>Store class name for timeline store.</description>
-  <name>yarn.timeline-service.store-class</name>
-  <value>org.apache.hadoop.yarn.server.timeline.LeveldbTimelineStore</value>
-</property>
-
-<property>
-  <description>Enable age off of timeline store data.</description>
-  <name>yarn.timeline-service.ttl-enable</name>
-  <value>true</value>
-</property>
-
-<property>
-  <description>Time to live for timeline store data in milliseconds.</description>
-  <name>yarn.timeline-service.ttl-ms</name>
-  <value>604800000</value>
-</property>
-+---+
-
-* Running Timeline server
-
-  Assuming all the aforementioned configurations are set properly, admins can
-  start the Timeline server/history service with the following command:
-
-+---+
-  $ yarn timelineserver
-+---+
-
-  Or users can start the Timeline server / history service as a daemon:
-
-+---+
-  $ yarn --daemon start timelineserver
-+---+
-
-* Accessing generic-data via command-line
-
-  Users can access applications' generic historic data via the command line as
-  below. Note that the same commands are usable to obtain the corresponding
-  information about running applications.
-
-+---+
-  $ yarn application -status <Application ID>
-  $ yarn applicationattempt -list <Application ID>
-  $ yarn applicationattempt -status <Application Attempt ID>
-  $ yarn container -list <Application Attempt ID>
-  $ yarn container -status <Container ID>
-+---+
-
-* Publishing of per-framework data by applications
-
-  Developers can define what information they want to record for their
-  applications by composing <<<TimelineEntity>>> and <<<TimelineEvent>>>
-  objects, and put the entities and events to the Timeline server via
-  <<<TimelineClient>>>. Below is an example:
-
-+---+
-  // Create and start the Timeline client
-  TimelineClient client = TimelineClient.createTimelineClient();
-  client.init(conf);
-  client.start();
-
-  TimelineEntity entity = null;
-  // Compose the entity
-  try {
-    TimelinePutResponse response = client.putEntities(entity);
-  } catch (IOException e) {
-    // Handle the exception
-  } catch (YarnException e) {
-    // Handle the exception
-  }
-
-  // Stop the Timeline client
-  client.stop();
-+---+

http://git-wip-us.apache.org/repos/asf/hadoop/blob/06aca7c6/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/apt/WebApplicationProxy.apt.vm
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/apt/WebApplicationProxy.apt.vm b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/apt/WebApplicationProxy.apt.vm
deleted file mode 100644
index 4646235..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/apt/WebApplicationProxy.apt.vm
+++ /dev/null
@@ -1,49 +0,0 @@
-~~ 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.
-
-  ---
-  YARN
-  ---
-  ---
-  ${maven.build.timestamp}
-
-Web Application Proxy
-
-  The Web Application Proxy is part of YARN.  By default it will run as part of
-  the Resource Manager(RM), but can be configured to run in stand alone mode.
-  The reason for the proxy is to reduce the possibility of web based attacks
-  through YARN.
-
-  In YARN the Application Master(AM) has the responsibility to provide a web UI
-  and to send that link to the RM.  This opens up a number of potential
-  issues.  The RM runs as a trusted user, and people visiting that web
-  address will treat it, and links it provides to them as trusted, when in
-  reality the AM is running as a non-trusted user, and the links it gives to
-  the RM could point to anything malicious or otherwise.  The Web Application
-  Proxy mitigates this risk by warning users that do not own the given
-  application that they are connecting to an untrusted site.
-
-  In addition to this the proxy also tries to reduce the impact that a malicious
-  AM could have on a user.  It primarily does this by stripping out cookies from
-  the user, and replacing them with a single cookie providing the user name of
-  the logged in user.  This is because most web based authentication systems will
-  identify a user based off of a cookie.  By providing this cookie to an
-  untrusted application it opens up the potential for an exploit.  If the cookie
-  is designed properly that potential should be fairly minimal, but this is just
-  to reduce that potential attack vector.  The current proxy implementation does
-  nothing to prevent the AM from providing links to malicious external sites,
-  nor does it do anything to prevent malicious javascript code from running as
-  well.  In fact javascript can be used to get the cookies, so stripping the
-  cookies from the request has minimal benefit at this time.
-
-  In the future we hope to address the attack vectors described above and make
-  attaching to an AM's web UI safer.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/06aca7c6/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/apt/WebServicesIntro.apt.vm
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/apt/WebServicesIntro.apt.vm b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/apt/WebServicesIntro.apt.vm
deleted file mode 100644
index 5300b94..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/apt/WebServicesIntro.apt.vm
+++ /dev/null
@@ -1,593 +0,0 @@
-~~ 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 YARN - Introduction to the web services REST API's.
-  ---
-  ---
-  ${maven.build.timestamp}
-
-Hadoop YARN - Introduction to the web services REST API's.
-
-%{toc|section=1|fromDepth=0}
-
-* Overview
-
-  The Hadoop YARN web service REST APIs are a set of URI resources that give access to the cluster, nodes, applications, and application historical information. The URI resources are grouped into APIs based on the type of information returned. Some URI resources return collections while others return singletons. 
-  
-* URI's
-
-  The URIs for the REST-based Web services have the following syntax:
-
-------
-  http://{http address of service}/ws/{version}/{resourcepath}
-------
-
-  The elements in this syntax are as follows:
-
-------
-  {http address of service} - The http address of the service to get information about. 
-                              Currently supported are the ResourceManager, NodeManager, 
-                              MapReduce application master, and history server.
-  {version} - The version of the APIs. In this release, the version is v1.
-  {resourcepath} - A path that defines a singleton resource or a collection of resources. 
-------
-
-* HTTP Requests
-
-  To invoke a REST API, your application calls an HTTP operation on the URI associated with a resource. 
-
-** Summary of HTTP operations
- 
-  Currently only GET is supported. It retrieves information about the resource specified.
-
-** Security
-
-  The web service REST API's go through the same security as the web ui.  If your cluster adminstrators have filters enabled you must authenticate via the mechanism they specified. 
-
-** Headers Supported
-
------
-  * Accept 
-  * Accept-Encoding
------
-
-  Currently the only fields used in the header is Accept and Accept-Encoding.  Accept currently supports XML and JSON for the response type you accept. Accept-Encoding currently only supports gzip format and will return gzip compressed output if this is specified, otherwise output is uncompressed. All other header fields are ignored.
-
-* HTTP Responses
-
-  The next few sections describe some of the syntax and other details of the HTTP Responses of the web service REST APIs.
-
-** Compression 
-
-  This release supports gzip compression if you specify gzip in the Accept-Encoding header of the HTTP request (Accept-Encoding: gzip).
-
-** Response Formats
-
-  This release of the web service REST APIs supports responses in JSON and XML formats. JSON is the default. To set the response format, you can specify the format in the Accept header of the HTTP request. 
-
-  As specified in HTTP Response Codes, the response body can contain the data that represents the resource or an error message. In the case of success, the response body is in the selected format, either JSON or XML. In the case of error, the resonse body is in either JSON or XML based on the format requested. The Content-Type header of the response contains the format requested. If the application requests an unsupported format, the response status code is 500.
-Note that the order of the fields within response body is not specified and might change. Also, additional fields might be added to a response body. Therefore, your applications should use parsing routines that can extract data from a response body in any order.
-
-** Response Errors
-
-  After calling an HTTP request, an application should check the response status code to verify success or detect an error. If the response status code indicates an error, the response body contains an error message. The first field is the exception type, currently only RemoteException is returned. The following table lists the items within the RemoteException error message:
-
-*---------------*--------------*-------------------------------*
-|| Item         || Data Type   || Description                  |
-*---------------+--------------+-------------------------------+
-| exception     | String       | Exception type                |
-*---------------+--------------+-------------------------------+
-| javaClassName | String       | Java class name of exception  |
-*---------------+--------------+-------------------------------+
-| message       | String       | Detailed message of exception |
-*---------------+--------------+-------------------------------+
-
-** Response Examples
-
-*** JSON response with single resource
-
-  HTTP Request:
-  GET http://rmhost.domain:8088/ws/v1/cluster/app/application_1324057493980_0001
-
-  Response Status Line:
-  HTTP/1.1 200 OK
-
-  Response Header:
-
-+---+
-  HTTP/1.1 200 OK
-  Content-Type: application/json
-  Transfer-Encoding: chunked
-  Server: Jetty(6.1.26)
-+---+
-
-  Response Body:
-
-+---+
-{
-  app":
-  {
-    "id":"application_1324057493980_0001",
-    "user":"user1",
-    "name":"",
-    "queue":"default",
-    "state":"ACCEPTED",
-    "finalStatus":"UNDEFINED",
-    "progress":0,
-    "trackingUI":"UNASSIGNED",
-    "diagnostics":"",
-    "clusterId":1324057493980,
-    "startedTime":1324057495921,
-    "finishedTime":0,
-    "elapsedTime":2063,
-    "amContainerLogs":"http:\/\/amNM:2\/node\/containerlogs\/container_1324057493980_0001_01_000001",
-    "amHostHttpAddress":"amNM:2"
-  }
-}
-+---+
-
-*** JSON response with Error response
-
-  Here we request information about an application that doesn't exist yet.
-
-  HTTP Request:
-  GET http://rmhost.domain:8088/ws/v1/cluster/app/application_1324057493980_9999
-
-  Response Status Line:
-  HTTP/1.1 404 Not Found
-
-  Response Header:
-
-+---+
-  HTTP/1.1 404 Not Found
-  Content-Type: application/json
-  Transfer-Encoding: chunked
-  Server: Jetty(6.1.26)
-+---+
-
-  Response Body:
-
-+---+
-{
-   "RemoteException" : {
-      "javaClassName" : "org.apache.hadoop.yarn.webapp.NotFoundException",
-      "exception" : "NotFoundException",
-      "message" : "java.lang.Exception: app with id: application_1324057493980_9999 not found"
-   }
-}
-+---+
-
-* Example usage
-
-  You can use any number of ways/languages to use the web services REST API's. This example uses the curl command line interface to do the REST GET calls.
-
-  In this example, a user submits a MapReduce application to the ResourceManager using a command like: 
-  
-+---+
-  hadoop jar hadoop-mapreduce-test.jar sleep -Dmapred.job.queue.name=a1 -m 1 -r 1 -rt 1200000 -mt 20
-+---+
-
-  The client prints information about the job submitted along with the application id, similar to:
-
-+---+
-12/01/18 04:25:15 INFO mapred.ResourceMgrDelegate: Submitted application application_1326821518301_0010 to ResourceManager at host.domain.com/10.10.10.10:8032
-12/01/18 04:25:15 INFO mapreduce.Job: Running job: job_1326821518301_0010
-12/01/18 04:25:21 INFO mapred.ClientServiceDelegate: The url to track the job: host.domain.com:8088/proxy/application_1326821518301_0010/
-12/01/18 04:25:22 INFO mapreduce.Job: Job job_1326821518301_0010 running in uber mode : false
-12/01/18 04:25:22 INFO mapreduce.Job:  map 0% reduce 0%
-+---+
-
-  The user then wishes to track the application. The users starts by getting the information about the application from the ResourceManager. Use the --comopressed option to request output compressed. curl handles uncompressing on client side.
-
-+---+
-curl --compressed -H "Accept: application/json" -X GET "http://host.domain.com:8088/ws/v1/cluster/apps/application_1326821518301_0010" 
-+---+
-
-  Output:
-
-+---+
-{
-   "app" : {
-      "finishedTime" : 0,
-      "amContainerLogs" : "http://host.domain.com:8042/node/containerlogs/container_1326821518301_0010_01_000001",
-      "trackingUI" : "ApplicationMaster",
-      "state" : "RUNNING",
-      "user" : "user1",
-      "id" : "application_1326821518301_0010",
-      "clusterId" : 1326821518301,
-      "finalStatus" : "UNDEFINED",
-      "amHostHttpAddress" : "host.domain.com:8042",
-      "progress" : 82.44703,
-      "name" : "Sleep job",
-      "startedTime" : 1326860715335,
-      "elapsedTime" : 31814,
-      "diagnostics" : "",
-      "trackingUrl" : "http://host.domain.com:8088/proxy/application_1326821518301_0010/",
-      "queue" : "a1"
-   }
-}
-+---+
-
-  The user then wishes to get more details about the running application and goes directly to the MapReduce application master for this application. The ResourceManager lists the trackingUrl that can be used for this application: http://host.domain.com:8088/proxy/application_1326821518301_0010. This could either go to the web browser or use the web service REST API's. The user uses the web services REST API's to get the list of jobs this MapReduce application master is running:
-
-+---+
- curl --compressed -H "Accept: application/json" -X GET "http://host.domain.com:8088/proxy/application_1326821518301_0010/ws/v1/mapreduce/jobs"
-+---+
-
-  Output:
-
-+---+
-{
-   "jobs" : {
-      "job" : [
-         {
-            "runningReduceAttempts" : 1,
-            "reduceProgress" : 72.104515,
-            "failedReduceAttempts" : 0,
-            "newMapAttempts" : 0,
-            "mapsRunning" : 0,
-            "state" : "RUNNING",
-            "successfulReduceAttempts" : 0,
-            "reducesRunning" : 1,
-            "acls" : [
-               {
-                  "value" : " ",
-                  "name" : "mapreduce.job.acl-modify-job"
-               },
-               {
-                  "value" : " ",
-                  "name" : "mapreduce.job.acl-view-job"
-               }
-            ],
-            "reducesPending" : 0,
-            "user" : "user1",
-            "reducesTotal" : 1,
-            "mapsCompleted" : 1,
-            "startTime" : 1326860720902,
-            "id" : "job_1326821518301_10_10",
-            "successfulMapAttempts" : 1,
-            "runningMapAttempts" : 0,
-            "newReduceAttempts" : 0,
-            "name" : "Sleep job",
-            "mapsPending" : 0,
-            "elapsedTime" : 64432,
-            "reducesCompleted" : 0,
-            "mapProgress" : 100,
-            "diagnostics" : "",
-            "failedMapAttempts" : 0,
-            "killedReduceAttempts" : 0,
-            "mapsTotal" : 1,
-            "uberized" : false,
-            "killedMapAttempts" : 0,
-            "finishTime" : 0
-         }
-      ]
-   }
-}
-+---+
-
-  The user then wishes to get the task details about the job with job id job_1326821518301_10_10 that was listed above. 
-
-+---+
- curl --compressed -H "Accept: application/json" -X GET "http://host.domain.com:8088/proxy/application_1326821518301_0010/ws/v1/mapreduce/jobs/job_1326821518301_10_10/tasks" 
-+---+
-
-  Output:
-
-+---+
-{
-   "tasks" : {
-      "task" : [
-         {
-            "progress" : 100,
-            "elapsedTime" : 5059,
-            "state" : "SUCCEEDED",
-            "startTime" : 1326860725014,
-            "id" : "task_1326821518301_10_10_m_0",
-            "type" : "MAP",
-            "successfulAttempt" : "attempt_1326821518301_10_10_m_0_0",
-            "finishTime" : 1326860730073
-         },
-         {
-            "progress" : 72.104515,
-            "elapsedTime" : 0,
-            "state" : "RUNNING",
-            "startTime" : 1326860732984,
-            "id" : "task_1326821518301_10_10_r_0",
-            "type" : "REDUCE",
-            "successfulAttempt" : "",
-            "finishTime" : 0
-         }
-      ]
-   }
-}
-+---+
-
-  The map task has finished but the reduce task is still running. The users wishes to get the task attempt information for the reduce task task_1326821518301_10_10_r_0, note that the Accept header isn't really required here since JSON is the default output format:
-
-+---+
-  curl --compressed -X GET "http://host.domain.com:8088/proxy/application_1326821518301_0010/ws/v1/mapreduce/jobs/job_1326821518301_10_10/tasks/task_1326821518301_10_10_r_0/attempts"
-+---+
-
-  Output:
-
-+---+
-{
-   "taskAttempts" : {
-      "taskAttempt" : [
-         {
-            "elapsedMergeTime" : 158,
-            "shuffleFinishTime" : 1326860735378,
-            "assignedContainerId" : "container_1326821518301_0010_01_000003",
-            "progress" : 72.104515,
-            "elapsedTime" : 0,
-            "state" : "RUNNING",
-            "elapsedShuffleTime" : 2394,
-            "mergeFinishTime" : 1326860735536,
-            "rack" : "/10.10.10.0",
-            "elapsedReduceTime" : 0,
-            "nodeHttpAddress" : "host.domain.com:8042",
-            "type" : "REDUCE",
-            "startTime" : 1326860732984,
-            "id" : "attempt_1326821518301_10_10_r_0_0",
-            "finishTime" : 0
-         }
-      ]
-   }
-}
-+---+
-
-  The reduce attempt is still running and the user wishes to see the current counter values for that attempt:
-
-+---+
- curl --compressed -H "Accept: application/json"  -X GET "http://host.domain.com:8088/proxy/application_1326821518301_0010/ws/v1/mapreduce/jobs/job_1326821518301_10_10/tasks/task_1326821518301_10_10_r_0/attempts/attempt_1326821518301_10_10_r_0_0/counters" 
-+---+
-
-  Output:
-
-+---+
-{
-   "JobTaskAttemptCounters" : {
-      "taskAttemptCounterGroup" : [
-         {
-            "counterGroupName" : "org.apache.hadoop.mapreduce.FileSystemCounter",
-            "counter" : [
-               {
-                  "value" : 4216,
-                  "name" : "FILE_BYTES_READ"
-               }, 
-               {
-                  "value" : 77151,
-                  "name" : "FILE_BYTES_WRITTEN"
-               }, 
-               {
-                  "value" : 0,
-                  "name" : "FILE_READ_OPS"
-               },
-               {
-                  "value" : 0,
-                  "name" : "FILE_LARGE_READ_OPS"
-               },
-               {
-                  "value" : 0,
-                  "name" : "FILE_WRITE_OPS"
-               },
-               {
-                  "value" : 0,
-                  "name" : "HDFS_BYTES_READ"
-               },
-               {
-                  "value" : 0,
-                  "name" : "HDFS_BYTES_WRITTEN"
-               },
-               {
-                  "value" : 0,
-                  "name" : "HDFS_READ_OPS"
-               },
-               {
-                  "value" : 0,
-                  "name" : "HDFS_LARGE_READ_OPS"
-               },
-               {
-                  "value" : 0,
-                  "name" : "HDFS_WRITE_OPS"
-               }
-            ]  
-         }, 
-         {
-            "counterGroupName" : "org.apache.hadoop.mapreduce.TaskCounter",
-            "counter" : [
-               {
-                  "value" : 0,
-                  "name" : "COMBINE_INPUT_RECORDS"
-               }, 
-               {
-                  "value" : 0,
-                  "name" : "COMBINE_OUTPUT_RECORDS"
-               }, 
-               {  
-                  "value" : 1767,
-                  "name" : "REDUCE_INPUT_GROUPS"
-               },
-               {  
-                  "value" : 25104,
-                  "name" : "REDUCE_SHUFFLE_BYTES"
-               },
-               {
-                  "value" : 1767,
-                  "name" : "REDUCE_INPUT_RECORDS"
-               },
-               {
-                  "value" : 0,
-                  "name" : "REDUCE_OUTPUT_RECORDS"
-               },
-               {
-                  "value" : 0,
-                  "name" : "SPILLED_RECORDS"
-               },
-               {
-                  "value" : 1,
-                  "name" : "SHUFFLED_MAPS"
-               },
-               {
-                  "value" : 0,
-                  "name" : "FAILED_SHUFFLE"
-               },
-               {
-                  "value" : 1,
-                  "name" : "MERGED_MAP_OUTPUTS"
-               },
-               {
-                  "value" : 50,
-                  "name" : "GC_TIME_MILLIS"
-               },
-               {
-                  "value" : 1580,
-                  "name" : "CPU_MILLISECONDS"
-               },
-               {
-                  "value" : 141320192,
-                  "name" : "PHYSICAL_MEMORY_BYTES"
-               },
-              {
-                  "value" : 1118552064,
-                  "name" : "VIRTUAL_MEMORY_BYTES"
-               }, 
-               {  
-                  "value" : 73728000,
-                  "name" : "COMMITTED_HEAP_BYTES"
-               }
-            ]
-         },
-         {  
-            "counterGroupName" : "Shuffle Errors",
-            "counter" : [
-               {  
-                  "value" : 0,
-                  "name" : "BAD_ID"
-               },
-               {  
-                  "value" : 0,
-                  "name" : "CONNECTION"
-               },
-               {  
-                  "value" : 0,
-                  "name" : "IO_ERROR"
-               },
-               {  
-                  "value" : 0,
-                  "name" : "WRONG_LENGTH"
-               },
-               {  
-                  "value" : 0,
-                  "name" : "WRONG_MAP"
-               },
-               {  
-                  "value" : 0,
-                  "name" : "WRONG_REDUCE"
-               }
-            ]
-         },
-         {  
-            "counterGroupName" : "org.apache.hadoop.mapreduce.lib.output.FileOutputFormatCounter",
-            "counter" : [
-              {  
-                  "value" : 0,
-                  "name" : "BYTES_WRITTEN"
-               }
-            ]
-         }
-      ],
-      "id" : "attempt_1326821518301_10_10_r_0_0"
-   }
-}
-+---+
-
-  The job finishes and the user wishes to get the final job information from the history server for this job.  
-
-+---+
-  curl --compressed -X GET "http://host.domain.com:19888/ws/v1/history/mapreduce/jobs/job_1326821518301_10_10" 
-+---+
-
-  Output:
-
-+---+
-{
-   "job" : {
-      "avgReduceTime" : 1250784,
-      "failedReduceAttempts" : 0,
-      "state" : "SUCCEEDED",
-      "successfulReduceAttempts" : 1,
-      "acls" : [
-         {
-            "value" : " ",
-            "name" : "mapreduce.job.acl-modify-job"
-         },
-         {
-            "value" : " ",
-            "name" : "mapreduce.job.acl-view-job"
-         }
-      ],
-      "user" : "user1",
-      "reducesTotal" : 1,
-      "mapsCompleted" : 1,
-      "startTime" : 1326860720902,
-      "id" : "job_1326821518301_10_10",
-      "avgMapTime" : 5059,
-      "successfulMapAttempts" : 1,
-      "name" : "Sleep job",
-      "avgShuffleTime" : 2394,
-      "reducesCompleted" : 1,
-      "diagnostics" : "",
-      "failedMapAttempts" : 0,
-      "avgMergeTime" : 2552,
-      "killedReduceAttempts" : 0,
-      "mapsTotal" : 1,
-      "queue" : "a1",
-      "uberized" : false,
-      "killedMapAttempts" : 0,
-      "finishTime" : 1326861986164
-   }
-}
-+---+
-
-  The user also gets the final applications information from the ResourceManager.
-
-+---+
-  curl --compressed -H "Accept: application/json" -X GET "http://host.domain.com:8088/ws/v1/cluster/apps/application_1326821518301_0010" 
-+---+
-
-  Output:
-
-+---+
-{
-   "app" : {
-      "finishedTime" : 1326861991282,
-      "amContainerLogs" : "http://host.domain.com:8042/node/containerlogs/container_1326821518301_0010_01_000001",
-      "trackingUI" : "History",
-      "state" : "FINISHED",
-      "user" : "user1",
-      "id" : "application_1326821518301_0010",
-      "clusterId" : 1326821518301,
-      "finalStatus" : "SUCCEEDED",
-      "amHostHttpAddress" : "host.domain.com:8042",
-      "progress" : 100,
-      "name" : "Sleep job",
-      "startedTime" : 1326860715335,
-      "elapsedTime" : 1275947,
-      "diagnostics" : "",
-      "trackingUrl" : "http://host.domain.com:8088/proxy/application_1326821518301_0010/jobhistory/job/job_1326821518301_10_10",
-      "queue" : "a1"
-   }
-}
-+---+


[35/50] [abbrv] hadoop git commit: YARN-3168. Convert site documentation from apt to markdown (Gururaj Shetty via aw)

Posted by zh...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/06aca7c6/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/apt/ResourceManagerRest.apt.vm
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/apt/ResourceManagerRest.apt.vm b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/apt/ResourceManagerRest.apt.vm
deleted file mode 100644
index 69728fb..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/apt/ResourceManagerRest.apt.vm
+++ /dev/null
@@ -1,3104 +0,0 @@
-~~ 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.
-
-  ---
-  ResourceManager REST API's.
-  ---
-  ---
-  ${maven.build.timestamp}
-
-ResourceManager REST API's.
-
-%{toc|section=1|fromDepth=0|toDepth=2}
-
-* Overview
-
-  The ResourceManager REST API's allow the user to get information about the cluster - status on the cluster, metrics on the cluster, scheduler information, information about nodes in the cluster, and information about applications on the cluster.
-  
-* Cluster Information API
-
-  The cluster information resource provides overall information about the cluster. 
-
-** URI
-
-  Both of the following URI's give you the cluster information.
-
-------
-  * http://<rm http address:port>/ws/v1/cluster
-  * http://<rm http address:port>/ws/v1/cluster/info
-------
-
-** HTTP Operations Supported
-
-------
-  * GET
-------
-
-** Query Parameters Supported
-
-------
-  None
-------
-
-** Elements of the <clusterInfo> object
-
-*---------------+--------------+-------------------------------+
-|| Item         || Data Type   || Description                   |
-*---------------+--------------+-------------------------------+
-| id            | long         | The cluster id |
-*---------------+--------------+-------------------------------+
-| startedOn     | long         | The time the cluster started (in ms since epoch)|
-*---------------+--------------+-------------------------------+
-| state         | string | The ResourceManager state - valid values are: NOTINITED, INITED, STARTED, STOPPED|
-*---------------+--------------+-------------------------------+
-| haState       | string | The ResourceManager HA state - valid values are: INITIALIZING, ACTIVE, STANDBY, STOPPED|
-*---------------+--------------+-------------------------------+
-| resourceManagerVersion | string  | Version of the ResourceManager |
-*---------------+--------------+-------------------------------+
-| resourceManagerBuildVersion | string  | ResourceManager build string with build version, user, and checksum |
-*---------------+--------------+-------------------------------+
-| resourceManagerVersionBuiltOn | string  | Timestamp when ResourceManager was built (in ms since epoch)|
-*---------------+--------------+-------------------------------+
-| hadoopVersion | string  | Version of hadoop common |
-*---------------+--------------+-------------------------------+
-| hadoopBuildVersion | string  | Hadoop common build string with build version, user, and checksum |
-*---------------+--------------+-------------------------------+
-| hadoopVersionBuiltOn | string  | Timestamp when hadoop common was built(in ms since epoch)|
-*---------------+--------------+-------------------------------+
-
-** Response Examples
-
-  <<JSON response>>
-
-  HTTP Request:
-
-------
-  GET http://<rm http address:port>/ws/v1/cluster/info
-------
-
-  Response Header:
-
-+---+
-  HTTP/1.1 200 OK
-  Content-Type: application/json
-  Transfer-Encoding: chunked
-  Server: Jetty(6.1.26)
-+---+
-
-  Response Body:
-
-+---+
-{
-  "clusterInfo":
-  {
-    "id":1324053971963,
-    "startedOn":1324053971963,
-    "state":"STARTED",
-    "resourceManagerVersion":"0.23.1-SNAPSHOT",
-    "resourceManagerBuildVersion":"0.23.1-SNAPSHOT from 1214049 by user1 source checksum 050cd664439d931c8743a6428fd6a693",
-    "resourceManagerVersionBuiltOn":"Tue Dec 13 22:12:48 CST 2011",
-    "hadoopVersion":"0.23.1-SNAPSHOT",
-    "hadoopBuildVersion":"0.23.1-SNAPSHOT from 1214049 by user1 source checksum 11458df3bb77342dca5f917198fad328",
-    "hadoopVersionBuiltOn":"Tue Dec 13 22:12:26 CST 2011"
-  }
-}
-+---+
-
-  <<XML response>>
-
-  HTTP Request:
-
------
-  Accept: application/xml
-  GET http://<rm http address:port>/ws/v1/cluster/info
------
-
-  Response Header:
-
-+---+
-  HTTP/1.1 200 OK
-  Content-Type: application/xml
-  Content-Length: 712
-  Server: Jetty(6.1.26)
-+---+
-
-  Response Body:
-
-+---+
-<?xml version="1.0" encoding="UTF-8" standalone="yes"?>
-<clusterInfo>
-  <id>1324053971963</id>
-  <startedOn>1324053971963</startedOn>
-  <state>STARTED</state>
-  <resourceManagerVersion>0.23.1-SNAPSHOT</resourceManagerVersion>
-  <resourceManagerBuildVersion>0.23.1-SNAPSHOT from 1214049 by user1 source checksum 050cd664439d931c8743a6428fd6a693</resourceManagerBuildVersion>
-  <resourceManagerVersionBuiltOn>Tue Dec 13 22:12:48 CST 2011</resourceManagerVersionBuiltOn>
-  <hadoopVersion>0.23.1-SNAPSHOT</hadoopVersion>
-  <hadoopBuildVersion>0.23.1-SNAPSHOT from 1214049 by user1 source checksum 11458df3bb77342dca5f917198fad328</hadoopBuildVersion>
-  <hadoopVersionBuiltOn>Tue Dec 13 22:12:48 CST 2011</hadoopVersionBuiltOn>
-</clusterInfo>
-+---+
-
-* Cluster Metrics API
-
-  The cluster metrics resource provides some overall metrics about the cluster. More detailed metrics should be retrieved from the jmx interface.
-
-** URI
-
-------
-  * http://<rm http address:port>/ws/v1/cluster/metrics
-------
-
-** HTTP Operations Supported 
-
-------
-  * GET
-------
-
-** Query Parameters Supported
-
-------
-  None
-------
-
-** Elements of the <clusterMetrics> object
-
-*---------------+--------------+-------------------------------+
-|| Item         || Data Type  || Description                   |
-*---------------+--------------+-------------------------------+
-| appsSubmitted | int | The number of applications submitted |
-*---------------+--------------+-------------------------------+
-| appsCompleted | int | The number of applications completed |
-*---------------+--------------+-------------------------------+
-| appsPending | int | The number of applications pending |
-*---------------+--------------+-------------------------------+
-| appsRunning | int | The number of applications running |
-*---------------+--------------+-------------------------------+
-| appsFailed | int | The number of applications failed |
-*---------------+--------------+-------------------------------+
-| appsKilled | int | The number of applications killed |
-*---------------+--------------+-------------------------------+
-| reservedMB    | long         | The amount of memory reserved in MB |
-*---------------+--------------+-------------------------------+
-| availableMB   | long         | The amount of memory available in MB |
-*---------------+--------------+-------------------------------+
-| allocatedMB   | long         | The amount of memory allocated in MB |
-*---------------+--------------+-------------------------------+
-| totalMB       | long         | The amount of total memory in MB |
-*---------------+--------------+-------------------------------+
-| reservedVirtualCores    | long         | The number of reserved virtual cores |
-*---------------+--------------+-------------------------------+
-| availableVirtualCores   | long         | The number of available virtual cores |
-*---------------+--------------+-------------------------------+
-| allocatedVirtualCores   | long         | The number of allocated virtual cores |
-*---------------+--------------+-------------------------------+
-| totalVirtualCores       | long         | The total number of virtual cores |
-*---------------+--------------+-------------------------------+
-| containersAllocated | int | The number of containers allocated |
-*---------------+--------------+-------------------------------+
-| containersReserved | int | The number of containers reserved |
-*---------------+--------------+-------------------------------+
-| containersPending | int | The number of containers pending |
-*---------------+--------------+-------------------------------+
-| totalNodes | int | The total number of nodes |
-*---------------+--------------+-------------------------------+
-| activeNodes | int | The number of active nodes |
-*---------------+--------------+-------------------------------+
-| lostNodes | int | The number of lost nodes |
-*---------------+--------------+-------------------------------+
-| unhealthyNodes | int | The number of unhealthy nodes |
-*---------------+--------------+-------------------------------+
-| decommissionedNodes | int | The number of nodes decommissioned |
-*---------------+--------------+-------------------------------+
-| rebootedNodes | int | The number of nodes rebooted |
-*---------------+--------------+-------------------------------+
-
-** Response Examples
-
-  <<JSON response>>
-
-  HTTP Request:
-
-------
-  GET http://<rm http address:port>/ws/v1/cluster/metrics
-------
-
-  Response Header:
-
-+---+
-  HTTP/1.1 200 OK
-  Content-Type: application/json
-  Transfer-Encoding: chunked
-  Server: Jetty(6.1.26)
-+---+
-
-  Response Body:
-
-+---+
-  {
-    "clusterMetrics":{
-      "appsSubmitted":0,
-      "appsCompleted":0,
-      "appsPending":0,
-      "appsRunning":0,
-      "appsFailed":0,
-      "appsKilled":0,
-      "reservedMB":0,
-      "availableMB":17408,
-      "allocatedMB":0,
-      "reservedVirtualCores":0,
-      "availableVirtualCores":7,
-      "allocatedVirtualCores":1,
-      "containersAllocated":0,
-      "containersReserved":0,
-      "containersPending":0,
-      "totalMB":17408,
-      "totalVirtualCores":8,
-      "totalNodes":1,
-      "lostNodes":0,
-      "unhealthyNodes":0,
-      "decommissionedNodes":0,
-      "rebootedNodes":0,
-      "activeNodes":1
-    }
-  }
-+---+
-
-  <<XML response>>
-
-  HTTP Request:
-
-------
-  GET http://<rm http address:port>/ws/v1/cluster/metrics
-  Accept: application/xml
-------
-
-  Response Header:
-
-+---+
-  HTTP/1.1 200 OK
-  Content-Type: application/xml
-  Content-Length: 432
-  Server: Jetty(6.1.26)
-+---+
-
-  Response Body:
-
-+---+
-<?xml version="1.0" encoding="UTF-8" standalone="yes"?>
-<clusterMetrics>
-  <appsSubmitted>0</appsSubmitted>
-  <appsCompleted>0</appsCompleted>
-  <appsPending>0</appsPending>
-  <appsRunning>0</appsRunning>
-  <appsFailed>0</appsFailed>
-  <appsKilled>0</appsKilled>
-  <reservedMB>0</reservedMB>
-  <availableMB>17408</availableMB>
-  <allocatedMB>0</allocatedMB>
-  <reservedVirtualCores>0</reservedVirtualCores>
-  <availableVirtualCores>7</availableVirtualCores>
-  <allocatedVirtualCores>1</allocatedVirtualCores>
-  <containersAllocated>0</containersAllocated>
-  <containersReserved>0</containersReserved>
-  <containersPending>0</containersPending>
-  <totalMB>17408</totalMB>
-  <totalVirtualCores>8</totalVirtualCores>
-  <totalNodes>1</totalNodes>
-  <lostNodes>0</lostNodes>
-  <unhealthyNodes>0</unhealthyNodes>
-  <decommissionedNodes>0</decommissionedNodes>
-  <rebootedNodes>0</rebootedNodes>
-  <activeNodes>1</activeNodes>
-</clusterMetrics>
-+---+
-
-* Cluster Scheduler API
-
-  A scheduler resource contains information about the current scheduler configured in a cluster. It currently supports both the Fifo and Capacity Scheduler. You will get different information depending on which scheduler is configured so be sure to look at the type information.
-
-** URI
-
-------
-  * http://<rm http address:port>/ws/v1/cluster/scheduler
-------
-
-** HTTP Operations Supported 
-
-------
-  * GET
-------
-
-** Query Parameters Supported
-
-------
-  None
-------
-
-** Capacity Scheduler API
-
-  The capacity scheduler supports hierarchical queues. This one request will print information about all the queues and any subqueues they have.
-  Queues that can actually have jobs submitted to them are referred to as leaf queues. These queues have additional data associated with them.
-
-** Elements of the <schedulerInfo> object
-
-*---------------+--------------+-------------------------------+
-|| Item          || Data Type  || Description                   |
-*---------------+--------------+-------------------------------+
-| type | string | Scheduler type - capacityScheduler|
-*---------------+--------------+-------------------------------+
-| capacity | float | Configured queue capacity in percentage relative to its parent queue |
-*---------------+--------------+-------------------------------+
-| usedCapacity | float | Used queue capacity in percentage |
-*---------------+--------------+-------------------------------+
-| maxCapacity | float | Configured maximum queue capacity in percentage relative to its parent queue|
-*---------------+--------------+-------------------------------+
-| queueName | string | Name of the queue |
-*---------------+--------------+-------------------------------+
-| queues | array of queues(JSON)/zero or more queue objects(XML) | A collection of queue resources|
-*---------------+--------------+-------------------------------+
-
-** Elements of the queues object for a Parent queue
-
-*---------------+--------------+-------------------------------+
-|| Item         || Data Type   || Description                   |
-*---------------+--------------+-------------------------------+
-| capacity | float | Configured queue capacity in percentage relative to its parent queue |
-*---------------+--------------+-------------------------------+
-| usedCapacity | float | Used queue capacity in percentage |
-*---------------+--------------+-------------------------------+
-| maxCapacity | float | Configured maximum queue capacity in percentage relative to its parent queue |
-*---------------+--------------+-------------------------------+
-| absoluteCapacity | float | Absolute capacity percentage this queue can use of entire cluster | 
-*---------------+--------------+-------------------------------+
-| absoluteMaxCapacity | float | Absolute maximum capacity percentage this queue can use of the entire cluster | 
-*---------------+--------------+-------------------------------+
-| absoluteUsedCapacity | float | Absolute used capacity percentage this queue is using of the entire cluster |
-*---------------+--------------+-------------------------------+
-| numApplications | int | The number of applications currently in the queue |
-*---------------+--------------+-------------------------------+
-| usedResources | string | A string describing the current resources used by the queue |
-*---------------+--------------+-------------------------------+
-| queueName | string | The name of the queue |
-*---------------+--------------+-------------------------------+
-| state | string of QueueState | The state of the queue |
-*---------------+--------------+-------------------------------+
-| queues | array of queues(JSON)/zero or more queue objects(XML) | A collection of sub-queue information|
-*---------------+--------------+-------------------------------+
-| resourcesUsed | A single resource object | The total amount of resources used by this queue |
-*---------------+--------------+-------------------------------+
-
-** Elements of the queues object for a Leaf queue - contains all elements in parent plus the following:
-
-*---------------+--------------+-------------------------------+
-|| Item         || Data Type   || Description                   |
-*---------------+--------------+-------------------------------+
-| type | String | type of the queue - capacitySchedulerLeafQueueInfo |
-*---------------+--------------+-------------------------------+
-| numActiveApplications | int | The number of active applications in this queue |
-*---------------+--------------+-------------------------------+
-| numPendingApplications | int | The number of pending applications in this queue |
-*---------------+--------------+-------------------------------+
-| numContainers | int | The number of containers being used |
-*---------------+--------------+-------------------------------+
-| maxApplications | int | The maximum number of applications this queue can have |
-*---------------+--------------+-------------------------------+
-| maxApplicationsPerUser | int | The maximum number of applications per user this queue can have |
-*---------------+--------------+-------------------------------+
-| maxActiveApplications | int | The maximum number of active applications this queue can have |
-*---------------+--------------+-------------------------------+
-| maxActiveApplicationsPerUser | int | The maximum number of active applications per user this queue can have|
-*---------------+--------------+-------------------------------+
-| userLimit | int | The minimum user limit percent set in the configuration |
-*---------------+--------------+-------------------------------+
-| userLimitFactor | float | The user limit factor set in the configuration |
-*---------------+--------------+-------------------------------+
-| users | array of users(JSON)/zero or more user objects(XML) | A collection of user objects containing resources used |
-*---------------+--------------+-------------------------------+
-
-** Elements of the user object for users:
-
-*---------------+--------------+-------------------------------+
-|| Item         || Data Type   || Description                   |
-*---------------+--------------+-------------------------------+
-| username | String | The username of the user using the resources |
-*---------------+--------------+-------------------------------+
-| resourcesUsed | A single resource object | The amount of resources used by the user in this queue |
-*---------------+--------------+-------------------------------+
-| numActiveApplications | int | The number of active applications for this user in this queue |
-*---------------+--------------+-------------------------------+
-| numPendingApplications | int | The number of pending applications for this user in this queue |
-*---------------+--------------+-------------------------------+
-
-** Elements of the resource object for resourcesUsed in user and queues:
-
-*---------------+--------------+-------------------------------+
-|| Item         || Data Type   || Description                   |
-*---------------+--------------+-------------------------------+
-| memory | int | The amount of memory used (in MB) |
-*---------------+--------------+-------------------------------+
-| vCores | int | The number of virtual cores |
-*---------------+--------------+-------------------------------+
-
-*** Response Examples
-
-  <<JSON response>>
-
-  HTTP Request:
-
-------
-  GET http://<rm http address:port>/ws/v1/cluster/scheduler
-------
-
-  Response Header:
-
-+---+
-  HTTP/1.1 200 OK
-  Content-Type: application/json
-  Transfer-Encoding: chunked
-  Server: Jetty(6.1.26)
-+---+
-
-  Response Body:
-
-+---+
-{
-    "scheduler": {
-        "schedulerInfo": {
-            "capacity": 100.0, 
-            "maxCapacity": 100.0, 
-            "queueName": "root", 
-            "queues": {
-                "queue": [
-                    {
-                        "absoluteCapacity": 10.5, 
-                        "absoluteMaxCapacity": 50.0, 
-                        "absoluteUsedCapacity": 0.0, 
-                        "capacity": 10.5, 
-                        "maxCapacity": 50.0, 
-                        "numApplications": 0, 
-                        "queueName": "a", 
-                        "queues": {
-                            "queue": [
-                                {
-                                    "absoluteCapacity": 3.15, 
-                                    "absoluteMaxCapacity": 25.0, 
-                                    "absoluteUsedCapacity": 0.0, 
-                                    "capacity": 30.000002, 
-                                    "maxCapacity": 50.0, 
-                                    "numApplications": 0, 
-                                    "queueName": "a1", 
-                                    "queues": {
-                                        "queue": [
-                                            {
-                                                "absoluteCapacity": 2.6775, 
-                                                "absoluteMaxCapacity": 25.0, 
-                                                "absoluteUsedCapacity": 0.0, 
-                                                "capacity": 85.0, 
-                                                "maxActiveApplications": 1, 
-                                                "maxActiveApplicationsPerUser": 1, 
-                                                "maxApplications": 267, 
-                                                "maxApplicationsPerUser": 267, 
-                                                "maxCapacity": 100.0, 
-                                                "numActiveApplications": 0, 
-                                                "numApplications": 0, 
-                                                "numContainers": 0, 
-                                                "numPendingApplications": 0, 
-                                                "queueName": "a1a", 
-                                                "resourcesUsed": {
-                                                    "memory": 0, 
-                                                    "vCores": 0
-                                                }, 
-                                                "state": "RUNNING", 
-                                                "type": "capacitySchedulerLeafQueueInfo", 
-                                                "usedCapacity": 0.0, 
-                                                "usedResources": "<memory:0, vCores:0>", 
-                                                "userLimit": 100, 
-                                                "userLimitFactor": 1.0, 
-                                                "users": null
-                                            }, 
-                                            {
-                                                "absoluteCapacity": 0.47250003, 
-                                                "absoluteMaxCapacity": 25.0, 
-                                                "absoluteUsedCapacity": 0.0, 
-                                                "capacity": 15.000001, 
-                                                "maxActiveApplications": 1, 
-                                                "maxActiveApplicationsPerUser": 1, 
-                                                "maxApplications": 47, 
-                                                "maxApplicationsPerUser": 47, 
-                                                "maxCapacity": 100.0, 
-                                                "numActiveApplications": 0, 
-                                                "numApplications": 0, 
-                                                "numContainers": 0, 
-                                                "numPendingApplications": 0, 
-                                                "queueName": "a1b", 
-                                                "resourcesUsed": {
-                                                    "memory": 0, 
-                                                    "vCores": 0
-                                                }, 
-                                                "state": "RUNNING", 
-                                                "type": "capacitySchedulerLeafQueueInfo", 
-                                                "usedCapacity": 0.0, 
-                                                "usedResources": "<memory:0, vCores:0>", 
-                                                "userLimit": 100, 
-                                                "userLimitFactor": 1.0, 
-                                                "users": null
-                                            }
-                                        ]
-                                    }, 
-                                    "resourcesUsed": {
-                                        "memory": 0, 
-                                        "vCores": 0
-                                    }, 
-                                    "state": "RUNNING", 
-                                    "usedCapacity": 0.0, 
-                                    "usedResources": "<memory:0, vCores:0>"
-                                }, 
-                                {
-                                    "absoluteCapacity": 7.35, 
-                                    "absoluteMaxCapacity": 50.0, 
-                                    "absoluteUsedCapacity": 0.0, 
-                                    "capacity": 70.0, 
-                                    "maxActiveApplications": 1, 
-                                    "maxActiveApplicationsPerUser": 100, 
-                                    "maxApplications": 735, 
-                                    "maxApplicationsPerUser": 73500, 
-                                    "maxCapacity": 100.0, 
-                                    "numActiveApplications": 0, 
-                                    "numApplications": 0, 
-                                    "numContainers": 0, 
-                                    "numPendingApplications": 0, 
-                                    "queueName": "a2", 
-                                    "resourcesUsed": {
-                                        "memory": 0, 
-                                        "vCores": 0
-                                    }, 
-                                    "state": "RUNNING", 
-                                    "type": "capacitySchedulerLeafQueueInfo", 
-                                    "usedCapacity": 0.0, 
-                                    "usedResources": "<memory:0, vCores:0>", 
-                                    "userLimit": 100, 
-                                    "userLimitFactor": 100.0, 
-                                    "users": null
-                                }
-                            ]
-                        }, 
-                        "resourcesUsed": {
-                            "memory": 0, 
-                            "vCores": 0
-                        }, 
-                        "state": "RUNNING", 
-                        "usedCapacity": 0.0, 
-                        "usedResources": "<memory:0, vCores:0>"
-                    }, 
-                    {
-                        "absoluteCapacity": 89.5, 
-                        "absoluteMaxCapacity": 100.0, 
-                        "absoluteUsedCapacity": 0.0, 
-                        "capacity": 89.5, 
-                        "maxCapacity": 100.0, 
-                        "numApplications": 2, 
-                        "queueName": "b", 
-                        "queues": {
-                            "queue": [
-                                {
-                                    "absoluteCapacity": 53.7, 
-                                    "absoluteMaxCapacity": 100.0, 
-                                    "absoluteUsedCapacity": 0.0, 
-                                    "capacity": 60.000004, 
-                                    "maxActiveApplications": 1, 
-                                    "maxActiveApplicationsPerUser": 100, 
-                                    "maxApplications": 5370, 
-                                    "maxApplicationsPerUser": 537000, 
-                                    "maxCapacity": 100.0, 
-                                    "numActiveApplications": 1, 
-                                    "numApplications": 2, 
-                                    "numContainers": 0, 
-                                    "numPendingApplications": 1, 
-                                    "queueName": "b1", 
-                                    "resourcesUsed": {
-                                        "memory": 0, 
-                                        "vCores": 0
-                                    }, 
-                                    "state": "RUNNING", 
-                                    "type": "capacitySchedulerLeafQueueInfo", 
-                                    "usedCapacity": 0.0, 
-                                    "usedResources": "<memory:0, vCores:0>", 
-                                    "userLimit": 100, 
-                                    "userLimitFactor": 100.0, 
-                                    "users": {
-                                        "user": [
-                                            {
-                                                "numActiveApplications": 0, 
-                                                "numPendingApplications": 1, 
-                                                "resourcesUsed": {
-                                                    "memory": 0, 
-                                                    "vCores": 0
-                                                }, 
-                                                "username": "user2"
-                                            }, 
-                                            {
-                                                "numActiveApplications": 1, 
-                                                "numPendingApplications": 0, 
-                                                "resourcesUsed": {
-                                                    "memory": 0, 
-                                                    "vCores": 0
-                                                }, 
-                                                "username": "user1"
-                                            }
-                                        ]
-                                    }
-                                }, 
-                                {
-                                    "absoluteCapacity": 35.3525, 
-                                    "absoluteMaxCapacity": 100.0, 
-                                    "absoluteUsedCapacity": 0.0, 
-                                    "capacity": 39.5, 
-                                    "maxActiveApplications": 1, 
-                                    "maxActiveApplicationsPerUser": 100, 
-                                    "maxApplications": 3535, 
-                                    "maxApplicationsPerUser": 353500, 
-                                    "maxCapacity": 100.0, 
-                                    "numActiveApplications": 0, 
-                                    "numApplications": 0, 
-                                    "numContainers": 0, 
-                                    "numPendingApplications": 0, 
-                                    "queueName": "b2", 
-                                    "resourcesUsed": {
-                                        "memory": 0, 
-                                        "vCores": 0
-                                    }, 
-                                    "state": "RUNNING", 
-                                    "type": "capacitySchedulerLeafQueueInfo", 
-                                    "usedCapacity": 0.0, 
-                                    "usedResources": "<memory:0, vCores:0>", 
-                                    "userLimit": 100, 
-                                    "userLimitFactor": 100.0, 
-                                    "users": null
-                                }, 
-                                {
-                                    "absoluteCapacity": 0.4475, 
-                                    "absoluteMaxCapacity": 100.0, 
-                                    "absoluteUsedCapacity": 0.0, 
-                                    "capacity": 0.5, 
-                                    "maxActiveApplications": 1, 
-                                    "maxActiveApplicationsPerUser": 100, 
-                                    "maxApplications": 44, 
-                                    "maxApplicationsPerUser": 4400, 
-                                    "maxCapacity": 100.0, 
-                                    "numActiveApplications": 0, 
-                                    "numApplications": 0, 
-                                    "numContainers": 0, 
-                                    "numPendingApplications": 0, 
-                                    "queueName": "b3", 
-                                    "resourcesUsed": {
-                                        "memory": 0, 
-                                        "vCores": 0
-                                    }, 
-                                    "state": "RUNNING", 
-                                    "type": "capacitySchedulerLeafQueueInfo", 
-                                    "usedCapacity": 0.0, 
-                                    "usedResources": "<memory:0, vCores:0>", 
-                                    "userLimit": 100, 
-                                    "userLimitFactor": 100.0, 
-                                    "users": null
-                                }
-                            ]
-                        }, 
-                        "resourcesUsed": {
-                            "memory": 0, 
-                            "vCores": 0
-                        }, 
-                        "state": "RUNNING", 
-                        "usedCapacity": 0.0, 
-                        "usedResources": "<memory:0, vCores:0>"
-                    }
-                ]
-            }, 
-            "type": "capacityScheduler", 
-            "usedCapacity": 0.0
-        }
-    }
-}
-+---+
-
-  <<XML response>>
-
-  HTTP Request:
-
------
-  Accept: application/xml
-  GET http://<rm http address:port>/ws/v1/cluster/scheduler
------
-
-  Response Header:
-
-+---+
-  HTTP/1.1 200 OK
-  Content-Type: application/xml
-  Content-Length: 5778
-  Server: Jetty(6.1.26)
-+---+
-
-  Response Body:
-
-+---+
-<?xml version="1.0" encoding="UTF-8" standalone="yes"?>
-<scheduler>
-  <schedulerInfo xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:type="capacityScheduler">
-    <capacity>100.0</capacity>
-    <usedCapacity>0.0</usedCapacity>
-    <maxCapacity>100.0</maxCapacity>
-    <queueName>root</queueName>
-    <queues>
-      <queue>
-        <capacity>10.5</capacity>
-        <usedCapacity>0.0</usedCapacity>
-        <maxCapacity>50.0</maxCapacity>
-        <absoluteCapacity>10.5</absoluteCapacity>
-        <absoluteMaxCapacity>50.0</absoluteMaxCapacity>
-        <absoluteUsedCapacity>0.0</absoluteUsedCapacity>
-        <numApplications>0</numApplications>
-        <usedResources>&lt;memory:0, vCores:0&gt;</usedResources>
-        <queueName>a</queueName>
-        <state>RUNNING</state>
-        <queues>
-          <queue>
-            <capacity>30.000002</capacity>
-            <usedCapacity>0.0</usedCapacity>
-            <maxCapacity>50.0</maxCapacity>
-            <absoluteCapacity>3.15</absoluteCapacity>
-            <absoluteMaxCapacity>25.0</absoluteMaxCapacity>
-            <absoluteUsedCapacity>0.0</absoluteUsedCapacity>
-            <numApplications>0</numApplications>
-            <usedResources>&lt;memory:0, vCores:0&gt;</usedResources>
-            <queueName>a1</queueName>
-            <state>RUNNING</state>
-            <queues>
-              <queue xsi:type="capacitySchedulerLeafQueueInfo">
-                <capacity>85.0</capacity>
-                <usedCapacity>0.0</usedCapacity>
-                <maxCapacity>100.0</maxCapacity>
-                <absoluteCapacity>2.6775</absoluteCapacity>
-                <absoluteMaxCapacity>25.0</absoluteMaxCapacity>
-                <absoluteUsedCapacity>0.0</absoluteUsedCapacity>
-                <numApplications>0</numApplications>
-                <usedResources>&lt;memory:0, vCores:0&gt;</usedResources>
-                <queueName>a1a</queueName>
-                <state>RUNNING</state>
-                <resourcesUsed>
-                  <memory>0</memory>
-                  <vCores>0</vCores>
-                </resourcesUsed>
-                <numActiveApplications>0</numActiveApplications>
-                <numPendingApplications>0</numPendingApplications>
-                <numContainers>0</numContainers>
-                <maxApplications>267</maxApplications>
-                <maxApplicationsPerUser>267</maxApplicationsPerUser>
-                <maxActiveApplications>1</maxActiveApplications>
-                <maxActiveApplicationsPerUser>1</maxActiveApplicationsPerUser>
-                <userLimit>100</userLimit>
-                <users/>
-                <userLimitFactor>1.0</userLimitFactor>
-              </queue>
-              <queue xsi:type="capacitySchedulerLeafQueueInfo">
-                <capacity>15.000001</capacity>
-                <usedCapacity>0.0</usedCapacity>
-                <maxCapacity>100.0</maxCapacity>
-                <absoluteCapacity>0.47250003</absoluteCapacity>
-                <absoluteMaxCapacity>25.0</absoluteMaxCapacity>
-                <absoluteUsedCapacity>0.0</absoluteUsedCapacity>
-                <numApplications>0</numApplications>
-                <usedResources>&lt;memory:0, vCores:0&gt;</usedResources>
-                <queueName>a1b</queueName>
-                <state>RUNNING</state>
-                <resourcesUsed>
-                  <memory>0</memory>
-                  <vCores>0</vCores>
-                </resourcesUsed>
-                <numActiveApplications>0</numActiveApplications>
-                <numPendingApplications>0</numPendingApplications>
-                <numContainers>0</numContainers>
-                <maxApplications>47</maxApplications>
-                <maxApplicationsPerUser>47</maxApplicationsPerUser>
-                <maxActiveApplications>1</maxActiveApplications>
-                <maxActiveApplicationsPerUser>1</maxActiveApplicationsPerUser>
-                <userLimit>100</userLimit>
-                <users/>
-                <userLimitFactor>1.0</userLimitFactor>
-              </queue>
-            </queues>
-            <resourcesUsed>
-              <memory>0</memory>
-              <vCores>0</vCores>
-            </resourcesUsed>
-          </queue>
-          <queue xsi:type="capacitySchedulerLeafQueueInfo">
-            <capacity>70.0</capacity>
-            <usedCapacity>0.0</usedCapacity>
-            <maxCapacity>100.0</maxCapacity>
-            <absoluteCapacity>7.35</absoluteCapacity>
-            <absoluteMaxCapacity>50.0</absoluteMaxCapacity>
-            <absoluteUsedCapacity>0.0</absoluteUsedCapacity>
-            <numApplications>0</numApplications>
-            <usedResources>&lt;memory:0, vCores:0&gt;</usedResources>
-            <queueName>a2</queueName>
-            <state>RUNNING</state>
-            <resourcesUsed>
-              <memory>0</memory>
-              <vCores>0</vCores>
-            </resourcesUsed>
-            <numActiveApplications>0</numActiveApplications>
-            <numPendingApplications>0</numPendingApplications>
-            <numContainers>0</numContainers>
-            <maxApplications>735</maxApplications>
-            <maxApplicationsPerUser>73500</maxApplicationsPerUser>
-            <maxActiveApplications>1</maxActiveApplications>
-            <maxActiveApplicationsPerUser>100</maxActiveApplicationsPerUser>
-            <userLimit>100</userLimit>
-            <users/>
-            <userLimitFactor>100.0</userLimitFactor>
-          </queue>
-        </queues>
-        <resourcesUsed>
-          <memory>0</memory>
-          <vCores>0</vCores>
-        </resourcesUsed>
-      </queue>
-      <queue>
-        <capacity>89.5</capacity>
-        <usedCapacity>0.0</usedCapacity>
-        <maxCapacity>100.0</maxCapacity>
-        <absoluteCapacity>89.5</absoluteCapacity>
-        <absoluteMaxCapacity>100.0</absoluteMaxCapacity>
-        <absoluteUsedCapacity>0.0</absoluteUsedCapacity>
-        <numApplications>2</numApplications>
-        <usedResources>&lt;memory:0, vCores:0&gt;</usedResources>
-        <queueName>b</queueName>
-        <state>RUNNING</state>
-        <queues>
-          <queue xsi:type="capacitySchedulerLeafQueueInfo">
-            <capacity>60.000004</capacity>
-            <usedCapacity>0.0</usedCapacity>
-            <maxCapacity>100.0</maxCapacity>
-            <absoluteCapacity>53.7</absoluteCapacity>
-            <absoluteMaxCapacity>100.0</absoluteMaxCapacity>
-            <absoluteUsedCapacity>0.0</absoluteUsedCapacity>
-            <numApplications>2</numApplications>
-            <usedResources>&lt;memory:0, vCores:0&gt;</usedResources>
-            <queueName>b1</queueName>
-            <state>RUNNING</state>
-            <resourcesUsed>
-              <memory>0</memory>
-              <vCores>0</vCores>
-            </resourcesUsed>
-            <numActiveApplications>1</numActiveApplications>
-            <numPendingApplications>1</numPendingApplications>
-            <numContainers>0</numContainers>
-            <maxApplications>5370</maxApplications>
-            <maxApplicationsPerUser>537000</maxApplicationsPerUser>
-            <maxActiveApplications>1</maxActiveApplications>
-            <maxActiveApplicationsPerUser>100</maxActiveApplicationsPerUser>
-            <userLimit>100</userLimit>
-            <users>
-              <user>
-                <username>user2</username>
-                <resourcesUsed>
-                  <memory>0</memory>
-                  <vCores>0</vCores>
-                </resourcesUsed>
-                <numPendingApplications>1</numPendingApplications>
-                <numActiveApplications>0</numActiveApplications>
-              </user>
-              <user>
-                <username>user1</username>
-                <resourcesUsed>
-                  <memory>0</memory>
-                  <vCores>0</vCores>
-                </resourcesUsed>
-                <numPendingApplications>0</numPendingApplications>
-                <numActiveApplications>1</numActiveApplications>
-              </user>
-            </users>
-            <userLimitFactor>100.0</userLimitFactor>
-          </queue>
-          <queue xsi:type="capacitySchedulerLeafQueueInfo">
-            <capacity>39.5</capacity>
-            <usedCapacity>0.0</usedCapacity>
-            <maxCapacity>100.0</maxCapacity>
-            <absoluteCapacity>35.3525</absoluteCapacity>
-            <absoluteMaxCapacity>100.0</absoluteMaxCapacity>
-            <absoluteUsedCapacity>0.0</absoluteUsedCapacity>
-            <numApplications>0</numApplications>
-            <usedResources>&lt;memory:0, vCores:0&gt;</usedResources>
-            <queueName>b2</queueName>
-            <state>RUNNING</state>
-            <resourcesUsed>
-              <memory>0</memory>
-              <vCores>0</vCores>
-            </resourcesUsed>
-            <numActiveApplications>0</numActiveApplications>
-            <numPendingApplications>0</numPendingApplications>
-            <numContainers>0</numContainers>
-            <maxApplications>3535</maxApplications>
-            <maxApplicationsPerUser>353500</maxApplicationsPerUser>
-            <maxActiveApplications>1</maxActiveApplications>
-            <maxActiveApplicationsPerUser>100</maxActiveApplicationsPerUser>
-            <userLimit>100</userLimit>
-            <users/>
-            <userLimitFactor>100.0</userLimitFactor>
-          </queue>
-          <queue xsi:type="capacitySchedulerLeafQueueInfo">
-            <capacity>0.5</capacity>
-            <usedCapacity>0.0</usedCapacity>
-            <maxCapacity>100.0</maxCapacity>
-            <absoluteCapacity>0.4475</absoluteCapacity>
-            <absoluteMaxCapacity>100.0</absoluteMaxCapacity>
-            <absoluteUsedCapacity>0.0</absoluteUsedCapacity>
-            <numApplications>0</numApplications>
-            <usedResources>&lt;memory:0, vCores:0&gt;</usedResources>
-            <queueName>b3</queueName>
-            <state>RUNNING</state>
-            <resourcesUsed>
-              <memory>0</memory>
-              <vCores>0</vCores>
-            </resourcesUsed>
-            <numActiveApplications>0</numActiveApplications>
-            <numPendingApplications>0</numPendingApplications>
-            <numContainers>0</numContainers>
-            <maxApplications>44</maxApplications>
-            <maxApplicationsPerUser>4400</maxApplicationsPerUser>
-            <maxActiveApplications>1</maxActiveApplications>
-            <maxActiveApplicationsPerUser>100</maxActiveApplicationsPerUser>
-            <userLimit>100</userLimit>
-            <users/>
-            <userLimitFactor>100.0</userLimitFactor>
-          </queue>
-        </queues>
-        <resourcesUsed>
-          <memory>0</memory>
-          <vCores>0</vCores>
-        </resourcesUsed>
-      </queue>
-    </queues>
-  </schedulerInfo>
-</scheduler>
-+---+
-
-** Fifo Scheduler API
-
-** Elements of the <schedulerInfo> object
-
-*---------------+--------------+-------------------------------+
-|| Item         || Data Type   || Description                   |
-*---------------+--------------+-------------------------------+
-| type | string | Scheduler type - fifoScheduler |
-*---------------+--------------+-------------------------------+
-| capacity | float | Queue capacity in percentage |
-*---------------+--------------+-------------------------------+
-| usedCapacity | float | Used queue capacity in percentage |
-*---------------+--------------+-------------------------------+
-| qstate | string | State of the queue - valid values are: STOPPED, RUNNING|
-*---------------+--------------+-------------------------------+
-| minQueueMemoryCapacity | int | Minimum queue memory capacity |
-*---------------+--------------+-------------------------------+
-| maxQueueMemoryCapacity | int | Maximum queue memory capacity |
-*---------------+--------------+-------------------------------+
-| numNodes | int | The total number of nodes |
-*---------------+--------------+-------------------------------+
-| usedNodeCapacity | int | The used node capacity |
-*---------------+--------------+-------------------------------+
-| availNodeCapacity | int | The available node capacity |
-*---------------+--------------+-------------------------------+
-| totalNodeCapacity | int | The total node capacity |
-*---------------+--------------+-------------------------------+
-| numContainers | int | The number of containers |
-*---------------+--------------+-------------------------------+
-
-*** Response Examples
-
-  <<JSON response>>
-
-  HTTP Request:
-
-------
-  GET http://<rm http address:port>/ws/v1/cluster/scheduler
-------
-
-  Response Header:
-
-+---+
-  HTTP/1.1 200 OK
-  Content-Type: application/json
-  Transfer-Encoding: chunked
-  Server: Jetty(6.1.26)
-+---+
-
-  Response Body:
-
-+---+
-{
-  "scheduler":
-  {
-    "schedulerInfo":
-    {
-      "type":"fifoScheduler",
-      "capacity":1,
-      "usedCapacity":"NaN",
-      "qstate":"RUNNING",
-      "minQueueMemoryCapacity":1024,
-      "maxQueueMemoryCapacity":10240,
-      "numNodes":0,
-      "usedNodeCapacity":0,
-      "availNodeCapacity":0,
-      "totalNodeCapacity":0,
-      "numContainers":0
-    }
-  }
-}
-+---+
-
-  <<JSON response>>
-
-  HTTP Request:
-
-------
-  GET http://<rm http address:port>/ws/v1/cluster/scheduler
-  Accept: application/xml
-------
-
-  Response Header:
-
-+---+
-  HTTP/1.1 200 OK
-  Content-Type: application/xml
-  Content-Length: 432
-  Server: Jetty(6.1.26)
-+---+
-
-  Response Body:
-
-+---+
-<?xml version="1.0" encoding="UTF-8" standalone="yes"?>
-<scheduler>
-  <schedulerInfo xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:type="fifoScheduler">
-    <capacity>1.0</capacity>
-    <usedCapacity>NaN</usedCapacity>
-    <qstate>RUNNING</qstate>
-    <minQueueMemoryCapacity>1024</minQueueMemoryCapacity>
-    <maxQueueMemoryCapacity>10240</maxQueueMemoryCapacity>
-    <numNodes>0</numNodes>
-    <usedNodeCapacity>0</usedNodeCapacity>
-    <availNodeCapacity>0</availNodeCapacity>
-    <totalNodeCapacity>0</totalNodeCapacity>
-    <numContainers>0</numContainers>
-  </schedulerInfo>
-</scheduler>
-+---+
-
-* {Cluster Applications API}
-
-  With the Applications API, you can obtain a collection of resources, each of which represents an application. When you run a GET operation on this resource, you obtain a collection of Application Objects.
-
-** URI
-
-------
-  * http://<rm http address:port>/ws/v1/cluster/apps
-------
-
-** HTTP Operations Supported 
-
-------
-  * GET
-------
-
-** Query Parameters Supported
-
-  Multiple parameters can be specified for GET operations.  The started and finished times have a begin and end parameter to allow you to specify ranges.  For example, one could request all applications that started between 1:00am and 2:00pm on 12/19/2011 with startedTimeBegin=1324256400&startedTimeEnd=1324303200. If the Begin parameter is not specified, it defaults to 0, and if the End parameter is not specified, it defaults to infinity.
-
-------
-  * state [deprecated] - state of the application
-  * states - applications matching the given application states, specified as a comma-separated list.
-  * finalStatus - the final status of the application - reported by the application itself
-  * user - user name
-  * queue - queue name
-  * limit - total number of app objects to be returned
-  * startedTimeBegin - applications with start time beginning with this time, specified in ms since epoch
-  * startedTimeEnd - applications with start time ending with this time, specified in ms since epoch
-  * finishedTimeBegin - applications with finish time beginning with this time, specified in ms since epoch
-  * finishedTimeEnd - applications with finish time ending with this time, specified in ms since epoch
-  * applicationTypes - applications matching the given application types, specified as a comma-separated list.
-  * applicationTags - applications matching any of the given application tags, specified as a comma-separated list.
-------
-
-** Elements of the <apps> (Applications) object
-
-  When you make a request for the list of applications, the information will be returned as a collection of app objects. 
-  See also {{Application API}} for syntax of the app object.
-
-*---------------+--------------+-------------------------------+
-|| Item         || Data Type   || Description                  |
-*---------------+--------------+-------------------------------+
-| app | array of app objects(JSON)/zero or more application objects(XML) | The collection of application objects |
-*---------------+--------------+--------------------------------+
-
-** Response Examples
-
-  <<JSON response>>
-
-  HTTP Request:
-
-------
-  GET http://<rm http address:port>/ws/v1/cluster/apps
-------
-
-  Response Header:
-
-+---+
-  HTTP/1.1 200 OK
-  Content-Type: application/json
-  Transfer-Encoding: chunked
-  Server: Jetty(6.1.26)
-+---+
-
-  Response Body:
-
-+---+
-{
-  "apps":
-  {
-    "app":
-    [
-       {
-          "finishedTime" : 1326815598530,
-          "amContainerLogs" : "http://host.domain.com:8042/node/containerlogs/container_1326815542473_0001_01_000001",
-          "trackingUI" : "History",
-          "state" : "FINISHED",
-          "user" : "user1",
-          "id" : "application_1326815542473_0001",
-          "clusterId" : 1326815542473,
-          "finalStatus" : "SUCCEEDED",
-          "amHostHttpAddress" : "host.domain.com:8042",
-          "progress" : 100,
-          "name" : "word count",
-          "startedTime" : 1326815573334,
-          "elapsedTime" : 25196,
-          "diagnostics" : "",
-          "trackingUrl" : "http://host.domain.com:8088/proxy/application_1326815542473_0001/jobhistory/job/job_1326815542473_1_1",
-          "queue" : "default",
-          "allocatedMB" : 0,
-          "allocatedVCores" : 0,
-          "runningContainers" : 0,
-          "memorySeconds" : 151730,
-          "vcoreSeconds" : 103
-       },
-       {
-          "finishedTime" : 1326815789546,
-          "amContainerLogs" : "http://host.domain.com:8042/node/containerlogs/container_1326815542473_0002_01_000001",
-          "trackingUI" : "History",
-          "state" : "FINISHED",
-          "user" : "user1",
-          "id" : "application_1326815542473_0002",
-          "clusterId" : 1326815542473,
-          "finalStatus" : "SUCCEEDED",
-          "amHostHttpAddress" : "host.domain.com:8042",
-          "progress" : 100,
-          "name" : "Sleep job",
-          "startedTime" : 1326815641380,
-          "elapsedTime" : 148166,
-          "diagnostics" : "",
-          "trackingUrl" : "http://host.domain.com:8088/proxy/application_1326815542473_0002/jobhistory/job/job_1326815542473_2_2",
-          "queue" : "default",
-          "allocatedMB" : 0,
-          "allocatedVCores" : 0,
-          "runningContainers" : 1,
-          "memorySeconds" : 640064,
-          "vcoreSeconds" : 442
-       } 
-    ]
-  }
-}
-+---+
-
-  <<JSON response>>
-
-  HTTP Request:
-
-------
-  GET http://<rm http address:port>/ws/v1/cluster/apps
-  Accept: application/xml
-------
-
-  Response Header:
-
-+---+
-  HTTP/1.1 200 OK
-  Content-Type: application/xml
-  Content-Length: 2459
-  Server: Jetty(6.1.26)
-+---+
-
-  Response Body:
-
-+---+
-<?xml version="1.0" encoding="UTF-8" standalone="yes"?>
-<apps>
-  <app>
-    <id>application_1326815542473_0001</id>
-    <user>user1</user>
-    <name>word count</name>
-    <applicationType>MAPREDUCE</applicationType>
-    <queue>default</queue>
-    <state>FINISHED</state>
-    <finalStatus>SUCCEEDED</finalStatus>
-    <progress>100.0</progress>
-    <trackingUI>History</trackingUI>
-    <trackingUrl>http://host.domain.com:8088/proxy/application_1326815542473_0001/jobhistory/job
-/job_1326815542473_1_1</trackingUrl>
-    <diagnostics/>
-    <clusterId>1326815542473</clusterId>
-    <startedTime>1326815573334</startedTime>
-    <finishedTime>1326815598530</finishedTime>
-    <elapsedTime>25196</elapsedTime>
-    <amContainerLogs>http://host.domain.com:8042/node/containerlogs/container_1326815542473_0001
-_01_000001</amContainerLogs>
-    <amHostHttpAddress>host.domain.com:8042</amHostHttpAddress>
-    <allocatedMB>0</allocatedMB>
-    <allocatedVCores>0</allocatedVCores>
-    <runningContainers>0</runningContainers>
-    <memorySeconds>151730</memorySeconds>
-    <vcoreSeconds>103</vcoreSeconds>
-  </app>
-  <app>
-    <id>application_1326815542473_0002</id>
-    <user>user1</user>
-    <name>Sleep job</name>
-    <applicationType>YARN</applicationType>
-    <queue>default</queue>
-    <state>FINISHED</state>
-    <finalStatus>SUCCEEDED</finalStatus>
-    <progress>100.0</progress>
-    <trackingUI>History</trackingUI>
-    <trackingUrl>http://host.domain.com:8088/proxy/application_1326815542473_0002/jobhistory/job/job_1326815542473_2_2</trackingUrl>
-    <diagnostics/>
-    <clusterId>1326815542473</clusterId>
-    <startedTime>1326815641380</startedTime>
-    <finishedTime>1326815789546</finishedTime>
-    <elapsedTime>148166</elapsedTime>
-    <amContainerLogs>http://host.domain.com:8042/node/containerlogs/container_1326815542473_0002_01_000001</amContainerLogs>
-    <amHostHttpAddress>host.domain.com:8042</amHostHttpAddress>
-    <allocatedMB>0</allocatedMB>
-    <allocatedVCores>0</allocatedVCores>
-    <runningContainers>0</runningContainers>
-    <memorySeconds>640064</memorySeconds>
-    <vcoreSeconds>442</vcoreSeconds>
-  </app>
-</apps>
-+---+
-
-* Cluster Application Statistics API
-
-  With the Application Statistics API, you can obtain a collection of triples, each of which contains the application type, the application state and the number of applications of this type and this state in ResourceManager context. Note that with the performance concern, we currently only support at most one applicationType per query. We may support multiple applicationTypes per query as well as more statistics in the future. When you run a GET operation on this resource, you obtain a collection of statItem objects. 
-
-** URI
-
-------
-  * http://<rm http address:port>/ws/v1/cluster/appstatistics
-------
-
-** HTTP Operations Supported
-
-------
-  * GET
-------
-
-** Query Parameters Required
-
-  Two paramters can be specified. The parameters are case insensitive.
-
-------
-  * states - states of the applications, specified as a comma-separated list. If states is not provided, the API will enumerate all application states and return the counts of them.
-  * applicationTypes - types of the applications, specified as a comma-separated list. If applicationTypes is not provided, the API will count the applications of any application type. In this case, the response shows * to indicate any application type. Note that we only support at most one applicationType temporarily. Otherwise, users will expect an BadRequestException.
-------
-
-** Elements of the <appStatInfo> (statItems) object
-
-  When you make a request for the list of statistics items, the information will be returned as a collection of statItem objects
-
-*-----------+----------------------------------------------------------------------+-------------------------------------+
-|| Item     || Data Type                                                           || Description                        |
-*-----------+----------------------------------------------------------------------+-------------------------------------+
-| statItem  | array of statItem objects(JSON)/zero or more statItem objects(XML)   | The collection of statItem objects  |
-*-----------+----------------------------------------------------------------------+-------------------------------------+
-
-** Response Examples
-
-  <<JSON response>>
-
-  HTTP Request:
-
-------
-  GET http://<rm http address:port>/ws/v1/cluster/appstatistics?states=accepted,running,finished&applicationTypes=mapreduce
-------
-
-  Response Header:
-
-+---+
-  HTTP/1.1 200 OK
-  Content-Type: application/json
-  Transfer-Encoding: chunked
-  Server: Jetty(6.1.26)
-+---+
-
-  Response Body:
-
-+---+
-{
-  "appStatInfo":
-  {
-    "statItem":
-    [
-       {
-          "state" : "accepted",
-          "type" : "mapreduce",
-          "count" : 4
-       },
-       {
-          "state" : "running",
-          "type" : "mapreduce",
-          "count" : 1
-       },
-       {
-          "state" : "finished",
-          "type" : "mapreduce",
-          "count" : 7
-       }
-    ]
-  }
-}
-+---+
-
-  <<XML response>>
-
-  HTTP Request:
-
-------
-  GET http://<rm http address:port>/ws/v1/cluster/appstatistics?states=accepted,running,finished&applicationTypes=mapreduce
-  Accept: application/xml
-------
-
-  Response Header:
-
-+---+
-  HTTP/1.1 200 OK
-  Content-Type: application/xml
-  Content-Length: 2459
-  Server: Jetty(6.1.26)
-+---+
-
-  Response Body:
-
-+---+
-<?xml version="1.0" encoding="UTF-8" standalone="yes"?>
-<appStatInfo>
-  <statItem>
-    <state>accepted</state>
-    <type>mapreduce</type>
-    <count>4</count>
-  </statItem>
-  <statItem>
-    <state>running</state>
-    <type>mapreduce</type>
-    <count>1</count>
-  </statItem>
-  <statItem>
-    <state>finished</state>
-    <type>mapreduce</type>
-    <count>7</count>
-  </statItem>
-</appStatInfo>
-+---+
-
-* Cluster {Application API}
-
-  An application resource contains information about a particular application that was submitted to a cluster.
-
-** URI
-
-  Use the following URI to obtain an app object, from a application identified by the {appid} value.
-
-------
-  * http://<rm http address:port>/ws/v1/cluster/apps/{appid}
-------
-
-** HTTP Operations Supported 
-
-------
-  * GET
-------
-
-** Query Parameters Supported
-
-------
-  None
-------
-
-** Elements of the <app> (Application) object
-
-  Note that depending on security settings a user might not be able to see all the fields. 
-
-*---------------+--------------+-------------------------------+
-|| Item         || Data Type   || Description                  |
-*---------------+--------------+-------------------------------+
-| id | string  | The application id | 
-*---------------+--------------+--------------------------------+
-| user | string  | The user who started the application |
-*---------------+--------------+--------------------------------+
-| name | string  | The application name |
-*---------------+--------------+--------------------------------+
-| Application Type | string  | The application type |
-*---------------+--------------+--------------------------------+
-| queue | string  | The queue the application was submitted to|
-*---------------+--------------+--------------------------------+
-| state         | string | The application state according to the ResourceManager - valid values are members of the YarnApplicationState enum: NEW, NEW_SAVING, SUBMITTED, ACCEPTED, RUNNING, FINISHED, FAILED, KILLED|
-*---------------+--------------+--------------------------------+
-| finalStatus | string | The final status of the application if finished - reported by the application itself - valid values are: UNDEFINED, SUCCEEDED, FAILED, KILLED|
-*---------------+--------------+--------------------------------+
-| progress | float | The progress of the application as a percent | 
-*---------------+--------------+--------------------------------+
-| trackingUI | string | Where the tracking url is currently pointing  - History (for history server) or ApplicationMaster |
-*---------------+--------------+--------------------------------+
-| trackingUrl | string | The web URL that can be used to track the application |
-*---------------+--------------+--------------------------------+
-| diagnostics | string | Detailed diagnostics information |
-*---------------+--------------+--------------------------------+
-| clusterId | long | The cluster id |
-*---------------+--------------+--------------------------------+
-| startedTime | long | The time in which application started (in ms since epoch)|
-*---------------+--------------+--------------------------------+
-| finishedTime | long | The time in which the application finished (in ms since epoch) |
-*---------------+--------------+--------------------------------+
-| elapsedTime | long | The elapsed time since the application started (in ms)|
-*---------------+--------------+--------------------------------+
-| amContainerLogs | string | The URL of the application master container logs|
-*---------------+--------------+--------------------------------+
-| amHostHttpAddress | string | The nodes http address of the application master |
-*---------------+--------------+--------------------------------+
-| allocatedMB | int | The sum of memory in MB allocated to the application's running containers |
-*---------------+--------------+--------------------------------+
-| allocatedVCores | int | The sum of virtual cores allocated to the application's running containers |
-*---------------+--------------+--------------------------------+
-| runningContainers | int | The number of containers currently running for the application |
-*---------------+--------------+--------------------------------+
-| memorySeconds | long | The amount of memory the application has allocated (megabyte-seconds) |
-*---------------+--------------+--------------------------------+
-| vcoreSeconds | long | The amount of CPU resources the application has allocated (virtual core-seconds) |
-*---------------+--------------+--------------------------------+
-
-** Response Examples
-
-  <<JSON response>>
-
-  HTTP Request:
-
-------
-  GET http://<rm http address:port>/ws/v1/cluster/apps/application_1326821518301_0005
-------
-
-  Response Header:
-
-+---+
-  HTTP/1.1 200 OK
-  Content-Type: application/json
-  Transfer-Encoding: chunked
-  Server: Jetty(6.1.26)
-+---+
-
-  Response Body:
-
-+---+
-{
-   "app" : {
-      "finishedTime" : 1326824991300,
-      "amContainerLogs" : "http://host.domain.com:8042/node/containerlogs/container_1326821518301_0005_01_000001",
-      "trackingUI" : "History",
-      "state" : "FINISHED",
-      "user" : "user1",
-      "id" : "application_1326821518301_0005",
-      "clusterId" : 1326821518301,
-      "finalStatus" : "SUCCEEDED",
-      "amHostHttpAddress" : "host.domain.com:8042",
-      "progress" : 100,
-      "name" : "Sleep job",
-      "applicationType" : "Yarn",
-      "startedTime" : 1326824544552,
-      "elapsedTime" : 446748,
-      "diagnostics" : "",
-      "trackingUrl" : "http://host.domain.com:8088/proxy/application_1326821518301_0005/jobhistory/job/job_1326821518301_5_5",
-      "queue" : "a1",
-      "memorySeconds" : 151730,
-      "vcoreSeconds" : 103
-   }
-}
-+---+
-
-  <<XML response>>
-
-  HTTP Request:
-
-------
-  GET http://<rm http address:port>/ws/v1/cluster/apps/application_1326821518301_0005
-  Accept: application/xml
-------
-
-  Response Header:
-
-+---+
-  HTTP/1.1 200 OK
-  Content-Type: application/xml
-  Content-Length: 847
-  Server: Jetty(6.1.26)
-+---+
-
-  Response Body:
-
-+---+
-<?xml version="1.0" encoding="UTF-8" standalone="yes"?>
-<app>
-  <id>application_1326821518301_0005</id>
-  <user>user1</user>
-  <name>Sleep job</name>
-  <queue>a1</queue>
-  <state>FINISHED</state>
-  <finalStatus>SUCCEEDED</finalStatus>
-  <progress>100.0</progress>
-  <trackingUI>History</trackingUI>
-  <trackingUrl>http://host.domain.com:8088/proxy/application_1326821518301_0005/jobhistory/job/job_1326821518301_5_5</trackingUrl>
-  <diagnostics/>
-  <clusterId>1326821518301</clusterId>
-  <startedTime>1326824544552</startedTime>
-  <finishedTime>1326824991300</finishedTime>
-  <elapsedTime>446748</elapsedTime>
-  <amContainerLogs>http://host.domain.com:8042/node/containerlogs/container_1326821518301_0005_01_000001</amContainerLogs>
-  <amHostHttpAddress>host.domain.com:8042</amHostHttpAddress>
-  <memorySeconds>151730</memorySeconds>
-  <vcoreSeconds>103</vcoreSeconds>
-</app>
-+---+
-
-* Cluster Application Attempts API
-
-  With the application attempts API, you can obtain a collection of resources that represent an application attempt.  When you run a GET operation on this resource, you obtain a collection of App Attempt Objects. 
-
-** URI
-
-------
-  * http://<rm http address:port>/ws/v1/cluster/apps/{appid}/appattempts
-------
-
-** HTTP Operations Supported 
-
-------
-  * GET
-------
-
-** Query Parameters Supported
-
-------
-  None
-------
-
-** Elements of the <appAttempts> object
-
-  When you make a request for the list of app attempts, the information will be returned as an array of app attempt objects. 
-
-  appAttempts:
-
-*---------------+--------------+-------------------------------+
-|| Item         || Data Type   || Description                  |
-*---------------+--------------+-------------------------------+
-| appAttempt | array of app attempt objects(JSON)/zero or more app attempt objects(XML) | The collection of app attempt objects |
-*---------------+--------------+--------------------------------+
-
-** Elements of the <appAttempt> object
-
-*---------------+--------------+-------------------------------+
-|| Item         || Data Type   || Description                  |
-*---------------+--------------+-------------------------------+
-| id | string | The app attempt id |
-*---------------+--------------+--------------------------------+
-| nodeId | string | The node id of the node the attempt ran on|
-*---------------+--------------+--------------------------------+
-| nodeHttpAddress | string | The node http address of the node the attempt ran on|
-*---------------+--------------+--------------------------------+
-| logsLink | string | The http link to the app attempt logs |
-*---------------+--------------+--------------------------------+
-| containerId | string | The id of the container for the app attempt |
-*---------------+--------------+--------------------------------+
-| startTime | long | The start time of the attempt (in ms since epoch)|
-*---------------+--------------+--------------------------------+
-
-** Response Examples
-
-  <<JSON response>>
-
-  HTTP Request:
-
-------
-  GET http://<rm http address:port>/ws/v1/cluster/apps/application_1326821518301_0005/appattempts
-------
-
-  Response Header:
-
-+---+
-  HTTP/1.1 200 OK
-  Content-Type: application/json
-  Transfer-Encoding: chunked
-  Server: Jetty(6.1.26)
-+---+
-
-  Response Body:
-
-+---+
-{
-   "appAttempts" : {
-      "appAttempt" : [
-         {
-            "nodeId" : "host.domain.com:8041",
-            "nodeHttpAddress" : "host.domain.com:8042",
-            "startTime" : 1326381444693,
-            "id" : 1,
-            "logsLink" : "http://host.domain.com:8042/node/containerlogs/container_1326821518301_0005_01_000001/user1",
-            "containerId" : "container_1326821518301_0005_01_000001"
-         }
-      ]
-   }
-}
-+---+
-
-  <<XML response>>
-
-  HTTP Request:
-
-------
-  GET http://<rm http address:port>/ws/v1/cluster/apps/application_1326821518301_0005/appattempts
-  Accept: application/xml
-------
-
-  Response Header:
-
-+---+
-  HTTP/1.1 200 OK
-  Content-Type: application/xml
-  Content-Length: 575
-  Server: Jetty(6.1.26)
-+---+
-
-  Response Body:
-
-+---+
-<?xml version="1.0" encoding="UTF-8" standalone="yes"?>
-<appAttempts>
-  <appttempt>
-    <nodeHttpAddress>host.domain.com:8042</nodeHttpAddress>
-    <nodeId>host.domain.com:8041</nodeId>
-    <id>1</id>
-    <startTime>1326381444693</startTime>
-    <containerId>container_1326821518301_0005_01_000001</containerId>
-    <logsLink>http://host.domain.com:8042/node/containerlogs/container_1326821518301_0005_01_000001/user1</logsLink>
-  </appAttempt>
-</appAttempts>
-+---+
-
-* Cluster Nodes API
-
-  With the Nodes API, you can obtain a collection of resources, each of which represents a node. When you run a GET operation on this resource, you obtain a collection of Node Objects. 
-
-** URI
-
-------
-  * http://<rm http address:port>/ws/v1/cluster/nodes
-------
-
-** HTTP Operations Supported 
-
-------
-  * GET
-------
-
-** Query Parameters Supported
-
-------
-  * state - the state of the node
-  * healthy - true or false 
-------
-
-** Elements of the <nodes> object
-
-  When you make a request for the list of nodes, the information will be returned as a collection of node objects. 
-  See also {{Node API}} for syntax of the node object.
-
-*---------------+--------------+-------------------------------+
-|| Item         || Data Type   || Description                   |
-*---------------+--------------+-------------------------------+
-| node | array of node objects(JSON)/zero or more node objects(XML) | A collection of node objects |
-*---------------+--------------+-------------------------------+
-
-** Response Examples
-
-  <<JSON response>>
-
-  HTTP Request:
-
-------
-  GET http://<rm http address:port>/ws/v1/cluster/nodes
-------
-
-  Response Header:
-
-+---+
-  HTTP/1.1 200 OK
-  Content-Type: application/json
-  Transfer-Encoding: chunked
-  Server: Jetty(6.1.26)
-+---+
-
-  Response Body:
-
-+---+
-{
-  "nodes":
-  {
-    "node":
-    [
-      {
-        "rack":"\/default-rack",
-        "state":"NEW",
-        "id":"h2:1235",
-        "nodeHostName":"h2",
-        "nodeHTTPAddress":"h2:2",
-        "healthStatus":"Healthy",
-        "lastHealthUpdate":1324056895432,
-        "healthReport":"Healthy",
-        "numContainers":0,
-        "usedMemoryMB":0,
-        "availMemoryMB":8192,
-        "usedVirtualCores":0,
-        "availableVirtualCores":8
-      },
-      {
-        "rack":"\/default-rack",
-        "state":"NEW",
-        "id":"h1:1234",
-        "nodeHostName":"h1",
-        "nodeHTTPAddress":"h1:2",
-        "healthStatus":"Healthy",
-        "lastHealthUpdate":1324056895092,
-        "healthReport":"Healthy",
-        "numContainers":0,
-        "usedMemoryMB":0,
-        "availMemoryMB":8192,
-        "usedVirtualCores":0,
-        "availableVirtualCores":8
-      }
-    ]
-  }
-}
-+---+
-
-  <<JSON response>>
-
-  HTTP Request:
-
-------
-  GET http://<rm http address:port>/ws/v1/cluster/nodes
-  Accept: application/xml
-------
-
-  Response Header:
-
-+---+
-  HTTP/1.1 200 OK
-  Content-Type: application/xml
-  Content-Length: 1104
-  Server: Jetty(6.1.26)
-+---+
-
-  Response Body:
-
-+---+
-<?xml version="1.0" encoding="UTF-8" standalone="yes"?>
-<nodes>
-  <node>
-    <rack>/default-rack</rack>
-    <state>RUNNING</state>
-    <id>h2:1234</id>
-    <nodeHostName>h2</nodeHostName>
-    <nodeHTTPAddress>h2:2</nodeHTTPAddress>
-    <healthStatus>Healthy</healthStatus>
-    <lastHealthUpdate>1324333268447</lastHealthUpdate>
-    <healthReport>Healthy</healthReport>
-    <numContainers>0</numContainers>
-    <usedMemoryMB>0</usedMemoryMB>
-    <availMemoryMB>5120</availMemoryMB>
-    <usedVirtualCores>0</usedVirtualCores>
-    <availableVirtualCores>8</availableVirtualCores>
-  </node>
-  <node>
-    <rack>/default-rack</rack>
-    <state>RUNNING</state>
-    <id>h1:1234</id>
-    <nodeHostName>h1</nodeHostName>
-    <nodeHTTPAddress>h1:2</nodeHTTPAddress>
-    <healthStatus>Healthy</healthStatus>
-    <lastHealthUpdate>1324333268447</lastHealthUpdate>
-    <healthReport>Healthy</healthReport>
-    <numContainers>0</numContainers>
-    <usedMemoryMB>0</usedMemoryMB>
-    <availMemoryMB>5120</availMemoryMB>
-    <usedVirtualCores>0</usedVirtualCores>
-    <availableVirtualCores>8</availableVirtualCores>
-  </node>
-</nodes>
-+---+
-
-
-* Cluster {Node API}
-
-  A node resource contains information about a node in the cluster.  
-
-** URI
-
-  Use the following URI to obtain a Node Object, from a node identified by the {nodeid} value. 
-
-------
-  * http://<rm http address:port>/ws/v1/cluster/nodes/{nodeid}
-------
-
-** HTTP Operations Supported 
-
-------
-  * GET
-------
-
-** Query Parameters Supported
-
-------
-  None
-------
-
-** Elements of the <node> object
-
-*---------------+--------------+-------------------------------+
-|| Item         || Data Type   || Description                   |
-*---------------+--------------+-------------------------------+
-| rack | string | The rack location of this node |
-*---------------+--------------+-------------------------------+
-| state | string | State of the node - valid values are: NEW, RUNNING, UNHEALTHY, DECOMMISSIONED, LOST, REBOOTED |
-*---------------+--------------+-------------------------------+
-| id | string  | The node id |
-*---------------+--------------+-------------------------------+
-| nodeHostName | string  | The host name of the node|
-*---------------+--------------+-------------------------------+
-| nodeHTTPAddress | string  | The nodes HTTP address|
-*---------------+--------------+-------------------------------+
-| healthStatus | string  | The health status of the node - Healthy or Unhealthy |
-*---------------+--------------+-------------------------------+
-| healthReport | string  | A detailed health report |
-*---------------+--------------+-------------------------------+
-| lastHealthUpdate | long | The last time the node reported its health (in ms since epoch)|
-*---------------+--------------+-------------------------------+
-| usedMemoryMB | long | The total amount of memory currently used on the node (in MB)|
-*---------------+--------------+-------------------------------+
-| availMemoryMB | long | The total amount of memory currently available on the node (in MB)|
-*---------------+--------------+-------------------------------+
-| usedVirtualCores | long | The total number of vCores currently used on the node |
-*---------------+--------------+-------------------------------+
-| availableVirtualCores | long | The total number of vCores available on the node |
-*---------------+--------------+-------------------------------+
-| numContainers | int | The total number of containers currently running on the node|
-*---------------+--------------+-------------------------------+
-
-** Response Examples
-
-  <<JSON response>>
-
-  HTTP Request:
-
-------
-  GET http://<rm http address:port>/ws/v1/cluster/nodes/h2:1235
-------
-
-  Response Header:
-
-+---+
-  HTTP/1.1 200 OK
-  Content-Type: application/json
-  Transfer-Encoding: chunked
-  Server: Jetty(6.1.26)
-+---+
-
-  Response Body:
-
-+---+
-{
-  "node":
-  {
-    "rack":"\/default-rack",
-    "state":"NEW",
-    "id":"h2:1235",
-    "nodeHostName":"h2",
-    "nodeHTTPAddress":"h2:2",
-    "healthStatus":"Healthy",
-    "lastHealthUpdate":1324056895432,
-    "healthReport":"Healthy",
-    "numContainers":0,
-    "usedMemoryMB":0,
-    "availMemoryMB":5120,
-    "usedVirtualCores":0,
-    "availableVirtualCores":8
-  }
-}
-+---+
-
-  <<JSON response>>
-
-  HTTP Request:
-
-------
-  GET http://<rm http address:port>/ws/v1/cluster/node/h2:1235
-  Accept: application/xml
-------
-
-  Response Header:
-
-+---+
-  HTTP/1.1 200 OK
-  Content-Type: application/xml
-  Content-Length: 552
-  Server: Jetty(6.1.26)
-+---+
-
-  Response Body:
-
-+---+
-<?xml version="1.0" encoding="UTF-8" standalone="yes"?>
-<node>
-  <rack>/default-rack</rack>
-  <state>NEW</state>
-  <id>h2:1235</id>
-  <nodeHostName>h2</nodeHostName>
-  <nodeHTTPAddress>h2:2</nodeHTTPAddress>
-  <healthStatus>Healthy</healthStatus>
-  <lastHealthUpdate>1324333268447</lastHealthUpdate>
-  <healthReport>Healthy</healthReport>
-  <numContainers>0</numContainers>
-  <usedMemoryMB>0</usedMemoryMB>
-  <availMemoryMB>5120</availMemoryMB>
-  <usedVirtualCores>0</usedVirtualCores>
-  <availableVirtualCores>5120</availableVirtualCores>
-</node>
-+---+
-
-* {Cluster Writeable APIs}
-
-  The setions below refer to APIs which allow to create and modify applications. These APIs are currently in alpha and may change in the future.
-
-* {Cluster New Application API}
-
-  With the New Application API, you can obtain an application-id which can then be used as part of the {{{Cluster_Applications_API(Submit_Application)}Cluster Submit Applications API}} to submit applications. The response also includes the maximum resource capabilities available on the cluster.
-
-   This feature is currently in the alpha stage and may change in the future.
-
-** URI
-
-------
-  * http://<rm http address:port>/ws/v1/cluster/apps/new-application
-------
-
-** HTTP Operations Supported
-
-------
-  * POST
-------
-
-** Query Parameters Supported
-
-------
-  * None
-------
-
-** Elements of the NewApplication object
-
-  The NewApplication response contains the following elements:
-
-*---------------+--------------+-------------------------------+
-|| Item         || Data Type   || Description                  |
-*---------------+--------------+-------------------------------+
-| application-id | string      | The newly created application id |
-*---------------+--------------+--------------------------------+
-| maximum-resource-capabilities | object  | The maximum resource capabilities available on this cluster |
-*---------------+--------------+--------------------------------+
-
-  The <maximum-resource-capabilites> object contains the following elements:
-
-
-*---------------+--------------+-------------------------------+
-|| Item         || Data Type   || Description                  |
-*---------------+--------------+-------------------------------+
-| memory        | int          | The maxiumim memory available for a container |
-*---------------+--------------+--------------------------------+
-| vCores        | int          | The maximum number of cores available for a container |
-*---------------+--------------+--------------------------------+
-
-** Response Examples
-
-  <<JSON response>>
-
-  HTTP Request:
-
-------
-  POST http://<rm http address:port>/ws/v1/cluster/apps/new-application
-------
-
-  Response Header:
-
-+---+
-  HTTP/1.1 200 OK
-  Content-Type: application/json
-  Transfer-Encoding: chunked
-  Server: Jetty(6.1.26)
-+---+
-
-  Response Body:
-
-+---+
-{
-  "application-id":"application_1404198295326_0003",
-  "maximum-resource-capability":
-    {
-      "memory":8192,
-      "vCores":32
-    }
-}
-+---+
-
-  <<XML response>>
-
-  HTTP Request:
-
-------
-  POST http://<rm http address:port>/ws/v1/cluster/apps/new-application
-------
-
-  Response Header:
-
-+---+
-  HTTP/1.1 200 OK
-  Content-Type: application/xml
-  Content-Length: 248
-  Server: Jetty(6.1.26)
-+---+
-
-  Response Body:
-
-+---+
-<?xml version="1.0" encoding="UTF-8" standalone="yes"?>
-<NewApplication>
-  <application-id>application_1404198295326_0003</application-id>
-  <maximum-resource-capability>
-    <memory>8192</memory>
-    <vCores>32</vCores>
-  </maximum-resource-capability>
-</NewApplication>
-+---+
-
-* {Cluster Applications API(Submit Application)}
-
-  The Submit Applications API can be used to submit applications. In case of submitting applications, you must first obtain an application-id using the {{{Cluster_New_Application_API}Cluster New Application API}}. The application-id must be part of the request body. The response contains a URL to the application page which can be used to track the state and progress of your application.
-
-** URI
-
-------
-  * http://<rm http address:port>/ws/v1/cluster/apps
-------
-
-** HTTP Operations Supported 
-
-------
-  * POST
-------
-
-** POST Response Examples
-
-  POST requests can be used to submit apps to the ResourceManager. As mentioned above, an application-id must be obtained first. Successful submissions result in a 202 response code and a Location header specifying where to get information about the app. Please note that in order to submit an app, you must have an authentication filter setup for the HTTP interface. The functionality requires that a username is set in the HttpServletRequest. If no filter is setup, the response will be an "UNAUTHORIZED" response.
-
-  Please note that this feature is currently in the alpha stage and may change in the future.
-
-*** Elements of the POST request object
-
-*---------------+--------------+-------------------------------+
-|| Item         || Data Type   || Description                  |
-*---------------+--------------+-------------------------------+
-| application-id | string      | The application id            |
-*---------------+--------------+-------------------------------+
-| application-name | string    | The application name          |
-*---------------+--------------+-------------------------------+
-| queue         | string       | The name of the queue to which the application should be submitted |
-*---------------+--------------+-------------------------------+
-| priority      | int          | The priority of the application |
-*---------------+--------------+-------------------------------+
-| am-container-spec | object   | The application master container launch context, described below |
-*---------------+--------------+-------------------------------+
-| unmanaged-AM  | boolean      | Is the application using an unmanaged application master |
-*---------------+--------------+-------------------------------+
-| max-app-attempts | int       | The max number of attempts for this application |
-*---------------+--------------+-------------------------------+
-| resource      | object       | The resources the application master requires, described below |
-*---------------+--------------+-------------------------------+
-| application-type | string    | The application type(MapReduce, Pig, Hive, etc) |
-*---------------+--------------+-------------------------------+
-| keep-containers-across-application-attempts | boolean | Should YARN keep the containers used by this application instead of destroying them |
-*---------------+--------------+-------------------------------+
-| application-tags | object    | List of application tags, please see the request examples on how to speciy the tags |
-*---------------+--------------+-------------------------------+
-
-  Elements of the <am-container-spec> object
-
-  The am-container-spec object should be used to provide the container launch context for the application master.
-
-*---------------+--------------+-------------------------------+
-|| Item         || Data Type   || Description                  |
-*---------------+--------------+-------------------------------+
-| local-resources | object     | Object describing the resources that need to be localized, described below |
-*---------------+--------------+-------------------------------+
-| environment   | object       | Environment variables for your containers, specified as key value pairs |
-*---------------+--------------+-------------------------------+
-| commands      | object       | The commands for launching your container, in the order in which they should be executed |
-*---------------+--------------+-------------------------------+
-| service-data  | object       | Application specific service data; key is the name of the auxiliary servce, value is base-64 encoding of the data you wish to pass |
-*---------------+--------------+-------------------------------+
-| credentials   | object       | The credentials required for your application to run, described below |
-*---------------+--------------+-------------------------------+
-| application-acls | objec     | ACLs for your application; the key can be "VIEW_APP" or "MODIFY_APP", the value is the list of users with the permissions |
-*---------------+--------------+-------------------------------+
-
-  Elements of the <local-resources> object
-
-  The object is a collection of key-value pairs. They key is an identifier for the resources to be localized and the value is the details of the resource. The elements of the value are described below:
-
-*---------------+--------------+-------------------------------+
-|| Item         || Data Type   || Description                  |
-*---------------+--------------+-------------------------------+
-| resource      | string       | Location of the resource to be localized |
-*---------------+--------------+-------------------------------+
-| type          | string       | Type of the resource; options are "ARCHIVE", "FILE", and "PATTERN" |
-*---------------+--------------+-------------------------------+
-| visibility    | string       | Visibility the resource to be localized; options are "PUBLIC", "PRIVATE", and "APPLICATION" |
-*---------------+--------------+-------------------------------+
-| size          | long         | Size of the resource to be localized |
-*---------------+--------------+-------------------------------+
-| timestamp     | long         | Timestamp of the resource to be localized |
-*---------------+--------------+-------------------------------+
-
-  Elements of the <credentials> object
-
-  The credentials object should be used to pass data required for the application to authenticate itself such as delegation-tokens and secrets.
-
-*---------------+--------------+-------------------------------+
-|| Item         || Data Type   || Description                  |
-*---------------+--------------+-------------------------------+
-| tokens        | object       | Tokens that you wish to pass to your application, specified as key-value pairs. The key is an identifier for the token and the value is the token(which should be obtained using the respective web-services) |
-*---------------+--------------+-------------------------------+
-| secrets       | object       | Secrets that you wish to use in your application, specified as key-value pairs. They key is an identifier and the value is the base-64 encoding of the secret |
-*---------------+--------------+-------------------------------+
-
-
-  Elements of the POST request body <resource> object
-
-*---------------+--------------+-------------------------------+
-|| Item         || Data Type   || Description                  |
-*---------------+--------------+-------------------------------+
-| memory        | int          | Memory required for each container |
-*---------------+--------------+-------------------------------+
-| vCores        | int          | Virtual cores required for each container |
-*---------------+--------------+-------------------------------+
-
-  <<JSON response>>
-
-  HTTP Request:
-
-+---+
-  POST http://<rm http address:port>/ws/v1/cluster/apps
-  Accept: application/json
-  Content-Type: application/json
-  {
-    "application-id":"application_1404203615263_0001",
-    "application-name":"test",
-    "am-container-spec":
-    {
-      "local-resources":
-      {
-        "entry":
-        [
-          {
-            "key":"AppMaster.jar",
-            "value":
-            {
-              "resource":"hdfs://hdfs-namenode:9000/user/testuser/DistributedShell/demo-app/AppMaster.jar",
-              "type":"FILE",
-              "visibility":"APPLICATION",
-              "size": 43004,
-              "ti

<TRUNCATED>

[31/50] [abbrv] hadoop git commit: YARN-3168. Convert site documentation from apt to markdown (Gururaj Shetty via aw)

Posted by zh...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/06aca7c6/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/ResourceManagerRest.md
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/ResourceManagerRest.md b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/ResourceManagerRest.md
new file mode 100644
index 0000000..b1591bb
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/ResourceManagerRest.md
@@ -0,0 +1,2640 @@
+<!---
+  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.
+-->
+
+ResourceManager REST API's.
+===========================
+
+* [Overview](#Overview)
+* [Cluster Information API](#Cluster_Information_API)
+* [Cluster Metrics API](#Cluster_Metrics_API)
+* [Cluster Scheduler API](#Cluster_Scheduler_API)
+* [Cluster Applications API](#Cluster_Applications_API)
+* [Cluster Application Statistics API](#Cluster_Application_Statistics_API)
+* [Cluster Application API](#Cluster_Application_API)
+* [Cluster Application Attempts API](#Cluster_Application_Attempts_API)
+* [Cluster Nodes API](#Cluster_Nodes_API)
+* [Cluster Node API](#Cluster_Node_API)
+* [Cluster Writeable APIs](#Cluster_Writeable_APIs)
+* [Cluster New Application API](#Cluster_New_Application_API)
+* [Cluster Applications API(Submit Application)](#Cluster_Applications_APISubmit_Application)
+* [Cluster Application State API](#Cluster_Application_State_API)
+* [Cluster Application Queue API](#Cluster_Application_Queue_API)
+* [Cluster Delegation Tokens API](#Cluster_Delegation_Tokens_API)
+
+Overview
+--------
+
+The ResourceManager REST API's allow the user to get information about the cluster - status on the cluster, metrics on the cluster, scheduler information, information about nodes in the cluster, and information about applications on the cluster.
+
+Cluster Information API
+-----------------------
+
+The cluster information resource provides overall information about the cluster.
+
+### URI
+
+Both of the following URI's give you the cluster information.
+
+      * http://<rm http address:port>/ws/v1/cluster
+      * http://<rm http address:port>/ws/v1/cluster/info
+
+### HTTP Operations Supported
+
+      * GET
+
+### Query Parameters Supported
+
+      None
+
+### Elements of the *clusterInfo* object
+
+| Item | Data Type | Description |
+|:---- |:---- |:---- |
+| id | long | The cluster id |
+| startedOn | long | The time the cluster started (in ms since epoch) |
+| state | string | The ResourceManager state - valid values are: NOTINITED, INITED, STARTED, STOPPED |
+| haState | string | The ResourceManager HA state - valid values are: INITIALIZING, ACTIVE, STANDBY, STOPPED |
+| resourceManagerVersion | string | Version of the ResourceManager |
+| resourceManagerBuildVersion | string | ResourceManager build string with build version, user, and checksum |
+| resourceManagerVersionBuiltOn | string | Timestamp when ResourceManager was built (in ms since epoch) |
+| hadoopVersion | string | Version of hadoop common |
+| hadoopBuildVersion | string | Hadoop common build string with build version, user, and checksum |
+| hadoopVersionBuiltOn | string | Timestamp when hadoop common was built(in ms since epoch) |
+
+### Response Examples
+
+**JSON response**
+
+HTTP Request:
+
+      GET http://<rm http address:port>/ws/v1/cluster/info
+
+Response Header:
+
+      HTTP/1.1 200 OK
+      Content-Type: application/json
+      Transfer-Encoding: chunked
+      Server: Jetty(6.1.26)
+
+Response Body:
+
+```json
+{
+  "clusterInfo":
+  {
+    "id":1324053971963,
+    "startedOn":1324053971963,
+    "state":"STARTED",
+    "resourceManagerVersion":"0.23.1-SNAPSHOT",
+    "resourceManagerBuildVersion":"0.23.1-SNAPSHOT from 1214049 by user1 source checksum 050cd664439d931c8743a6428fd6a693",
+    "resourceManagerVersionBuiltOn":"Tue Dec 13 22:12:48 CST 2011",
+    "hadoopVersion":"0.23.1-SNAPSHOT",
+    "hadoopBuildVersion":"0.23.1-SNAPSHOT from 1214049 by user1 source checksum 11458df3bb77342dca5f917198fad328",
+    "hadoopVersionBuiltOn":"Tue Dec 13 22:12:26 CST 2011"
+  }
+}
+```
+
+**XML response**
+
+HTTP Request:
+
+      Accept: application/xml
+      GET http://<rm http address:port>/ws/v1/cluster/info
+
+Response Header:
+
+      HTTP/1.1 200 OK
+      Content-Type: application/xml
+      Content-Length: 712
+      Server: Jetty(6.1.26)
+
+Response Body:
+
+```xml
+<?xml version="1.0" encoding="UTF-8" standalone="yes"?>
+<clusterInfo>
+  <id>1324053971963</id>
+  <startedOn>1324053971963</startedOn>
+  <state>STARTED</state>
+  <resourceManagerVersion>0.23.1-SNAPSHOT</resourceManagerVersion>
+  <resourceManagerBuildVersion>0.23.1-SNAPSHOT from 1214049 by user1 source checksum 050cd664439d931c8743a6428fd6a693</resourceManagerBuildVersion>
+  <resourceManagerVersionBuiltOn>Tue Dec 13 22:12:48 CST 2011</resourceManagerVersionBuiltOn>
+  <hadoopVersion>0.23.1-SNAPSHOT</hadoopVersion>
+  <hadoopBuildVersion>0.23.1-SNAPSHOT from 1214049 by user1 source checksum 11458df3bb77342dca5f917198fad328</hadoopBuildVersion>
+  <hadoopVersionBuiltOn>Tue Dec 13 22:12:48 CST 2011</hadoopVersionBuiltOn>
+</clusterInfo>
+```
+
+Cluster Metrics API
+-------------------
+
+The cluster metrics resource provides some overall metrics about the cluster. More detailed metrics should be retrieved from the jmx interface.
+
+### URI
+
+      * http://<rm http address:port>/ws/v1/cluster/metrics
+
+### HTTP Operations Supported
+
+      * GET
+
+### Query Parameters Supported
+
+      None
+
+### Elements of the *clusterMetrics* object
+
+| Item | Data Type | Description |
+|:---- |:---- |:---- |
+| appsSubmitted | int | The number of applications submitted |
+| appsCompleted | int | The number of applications completed |
+| appsPending | int | The number of applications pending |
+| appsRunning | int | The number of applications running |
+| appsFailed | int | The number of applications failed |
+| appsKilled | int | The number of applications killed |
+| reservedMB | long | The amount of memory reserved in MB |
+| availableMB | long | The amount of memory available in MB |
+| allocatedMB | long | The amount of memory allocated in MB |
+| totalMB | long | The amount of total memory in MB |
+| reservedVirtualCores | long | The number of reserved virtual cores |
+| availableVirtualCores | long | The number of available virtual cores |
+| allocatedVirtualCores | long | The number of allocated virtual cores |
+| totalVirtualCores | long | The total number of virtual cores |
+| containersAllocated | int | The number of containers allocated |
+| containersReserved | int | The number of containers reserved |
+| containersPending | int | The number of containers pending |
+| totalNodes | int | The total number of nodes |
+| activeNodes | int | The number of active nodes |
+| lostNodes | int | The number of lost nodes |
+| unhealthyNodes | int | The number of unhealthy nodes |
+| decommissionedNodes | int | The number of nodes decommissioned |
+| rebootedNodes | int | The number of nodes rebooted |
+
+### Response Examples
+
+**JSON response**
+
+HTTP Request:
+
+      GET http://<rm http address:port>/ws/v1/cluster/metrics
+
+Response Header:
+
+      HTTP/1.1 200 OK
+      Content-Type: application/json
+      Transfer-Encoding: chunked
+      Server: Jetty(6.1.26)
+
+Response Body:
+
+```json
+{
+  "clusterMetrics":
+  {
+    "appsSubmitted":0,
+    "appsCompleted":0,
+    "appsPending":0,
+    "appsRunning":0,
+    "appsFailed":0,
+    "appsKilled":0,
+    "reservedMB":0,
+    "availableMB":17408,
+    "allocatedMB":0,
+    "reservedVirtualCores":0,
+    "availableVirtualCores":7,
+    "allocatedVirtualCores":1,
+    "containersAllocated":0,
+    "containersReserved":0,
+    "containersPending":0,
+    "totalMB":17408,
+    "totalVirtualCores":8,
+    "totalNodes":1,
+    "lostNodes":0,
+    "unhealthyNodes":0,
+    "decommissionedNodes":0,
+    "rebootedNodes":0,
+    "activeNodes":1
+  }
+}
+```
+
+**XML response**
+
+HTTP Request:
+
+      GET http://<rm http address:port>/ws/v1/cluster/metrics
+      Accept: application/xml
+
+Response Header:
+
+      HTTP/1.1 200 OK
+      Content-Type: application/xml
+      Content-Length: 432
+      Server: Jetty(6.1.26)
+
+Response Body:
+
+```xml
+<?xml version="1.0" encoding="UTF-8" standalone="yes"?>
+<clusterMetrics>
+  <appsSubmitted>0</appsSubmitted>
+  <appsCompleted>0</appsCompleted>
+  <appsPending>0</appsPending>
+  <appsRunning>0</appsRunning>
+  <appsFailed>0</appsFailed>
+  <appsKilled>0</appsKilled>
+  <reservedMB>0</reservedMB>
+  <availableMB>17408</availableMB>
+  <allocatedMB>0</allocatedMB>
+  <reservedVirtualCores>0</reservedVirtualCores>
+  <availableVirtualCores>7</availableVirtualCores>
+  <allocatedVirtualCores>1</allocatedVirtualCores>
+  <containersAllocated>0</containersAllocated>
+  <containersReserved>0</containersReserved>
+  <containersPending>0</containersPending>
+  <totalMB>17408</totalMB>
+  <totalVirtualCores>8</totalVirtualCores>
+  <totalNodes>1</totalNodes>
+  <lostNodes>0</lostNodes>
+  <unhealthyNodes>0</unhealthyNodes>
+  <decommissionedNodes>0</decommissionedNodes>
+  <rebootedNodes>0</rebootedNodes>
+  <activeNodes>1</activeNodes>
+</clusterMetrics>
+```
+
+Cluster Scheduler API
+---------------------
+
+A scheduler resource contains information about the current scheduler configured in a cluster. It currently supports both the Fifo and Capacity Scheduler. You will get different information depending on which scheduler is configured so be sure to look at the type information.
+
+### URI
+
+      * http://<rm http address:port>/ws/v1/cluster/scheduler
+
+### HTTP Operations Supported
+
+      * GET
+
+### Query Parameters Supported
+
+      None
+
+### Capacity Scheduler API
+
+The capacity scheduler supports hierarchical queues. This one request will print information about all the queues and any subqueues they have. Queues that can actually have jobs submitted to them are referred to as leaf queues. These queues have additional data associated with them.
+
+### Elements of the *schedulerInfo* object
+
+| Item | Data Type | Description |
+|:---- |:---- |:---- |
+| type | string | Scheduler type - capacityScheduler |
+| capacity | float | Configured queue capacity in percentage relative to its parent queue |
+| usedCapacity | float | Used queue capacity in percentage |
+| maxCapacity | float | Configured maximum queue capacity in percentage relative to its parent queue |
+| queueName | string | Name of the queue |
+| queues | array of queues(JSON)/zero or more queue objects(XML) | A collection of queue resources |
+
+### Elements of the queues object for a Parent queue
+
+| Item | Data Type | Description |
+|:---- |:---- |:---- |
+| capacity | float | Configured queue capacity in percentage relative to its parent queue |
+| usedCapacity | float | Used queue capacity in percentage |
+| maxCapacity | float | Configured maximum queue capacity in percentage relative to its parent queue |
+| absoluteCapacity | float | Absolute capacity percentage this queue can use of entire cluster |
+| absoluteMaxCapacity | float | Absolute maximum capacity percentage this queue can use of the entire cluster |
+| absoluteUsedCapacity | float | Absolute used capacity percentage this queue is using of the entire cluster |
+| numApplications | int | The number of applications currently in the queue |
+| usedResources | string | A string describing the current resources used by the queue |
+| queueName | string | The name of the queue |
+| state | string of QueueState | The state of the queue |
+| queues | array of queues(JSON)/zero or more queue objects(XML) | A collection of sub-queue information |
+| resourcesUsed | A single resource object | The total amount of resources used by this queue |
+
+### Elements of the queues object for a Leaf queue - contains all elements in parent plus the following:
+
+| Item | Data Type | Description |
+|:---- |:---- |:---- |
+| type | String | type of the queue - capacitySchedulerLeafQueueInfo |
+| numActiveApplications | int | The number of active applications in this queue |
+| numPendingApplications | int | The number of pending applications in this queue |
+| numContainers | int | The number of containers being used |
+| maxApplications | int | The maximum number of applications this queue can have |
+| maxApplicationsPerUser | int | The maximum number of applications per user this queue can have |
+| maxActiveApplications | int | The maximum number of active applications this queue can have |
+| maxActiveApplicationsPerUser | int | The maximum number of active applications per user this queue can have |
+| userLimit | int | The minimum user limit percent set in the configuration |
+| userLimitFactor | float | The user limit factor set in the configuration |
+| users | array of users(JSON)/zero or more user objects(XML) | A collection of user objects containing resources used |
+
+### Elements of the user object for users:
+
+| Item | Data Type | Description |
+|:---- |:---- |:---- |
+| username | String | The username of the user using the resources |
+| resourcesUsed | A single resource object | The amount of resources used by the user in this queue |
+| numActiveApplications | int | The number of active applications for this user in this queue |
+| numPendingApplications | int | The number of pending applications for this user in this queue |
+
+### Elements of the resource object for resourcesUsed in user and queues:
+
+| Item | Data Type | Description |
+|:---- |:---- |:---- |
+| memory | int | The amount of memory used (in MB) |
+| vCores | int | The number of virtual cores |
+
+#### Response Examples
+
+**JSON response**
+
+HTTP Request:
+
+      GET http://<rm http address:port>/ws/v1/cluster/scheduler
+
+Response Header:
+
+      HTTP/1.1 200 OK
+      Content-Type: application/json
+      Transfer-Encoding: chunked
+      Server: Jetty(6.1.26)
+
+Response Body:
+
+```json
+{
+    "scheduler": {
+        "schedulerInfo": {
+            "capacity": 100.0, 
+            "maxCapacity": 100.0, 
+            "queueName": "root", 
+            "queues": {
+                "queue": [
+                    {
+                        "absoluteCapacity": 10.5, 
+                        "absoluteMaxCapacity": 50.0, 
+                        "absoluteUsedCapacity": 0.0, 
+                        "capacity": 10.5, 
+                        "maxCapacity": 50.0, 
+                        "numApplications": 0, 
+                        "queueName": "a", 
+                        "queues": {
+                            "queue": [
+                                {
+                                    "absoluteCapacity": 3.15, 
+                                    "absoluteMaxCapacity": 25.0, 
+                                    "absoluteUsedCapacity": 0.0, 
+                                    "capacity": 30.000002, 
+                                    "maxCapacity": 50.0, 
+                                    "numApplications": 0, 
+                                    "queueName": "a1", 
+                                    "queues": {
+                                        "queue": [
+                                            {
+                                                "absoluteCapacity": 2.6775, 
+                                                "absoluteMaxCapacity": 25.0, 
+                                                "absoluteUsedCapacity": 0.0, 
+                                                "capacity": 85.0, 
+                                                "maxActiveApplications": 1, 
+                                                "maxActiveApplicationsPerUser": 1, 
+                                                "maxApplications": 267, 
+                                                "maxApplicationsPerUser": 267, 
+                                                "maxCapacity": 100.0, 
+                                                "numActiveApplications": 0, 
+                                                "numApplications": 0, 
+                                                "numContainers": 0, 
+                                                "numPendingApplications": 0, 
+                                                "queueName": "a1a", 
+                                                "resourcesUsed": {
+                                                    "memory": 0, 
+                                                    "vCores": 0
+                                                }, 
+                                                "state": "RUNNING", 
+                                                "type": "capacitySchedulerLeafQueueInfo", 
+                                                "usedCapacity": 0.0, 
+                                                "usedResources": "<memory:0, vCores:0>", 
+                                                "userLimit": 100, 
+                                                "userLimitFactor": 1.0, 
+                                                "users": null
+                                            }, 
+                                            {
+                                                "absoluteCapacity": 0.47250003, 
+                                                "absoluteMaxCapacity": 25.0, 
+                                                "absoluteUsedCapacity": 0.0, 
+                                                "capacity": 15.000001, 
+                                                "maxActiveApplications": 1, 
+                                                "maxActiveApplicationsPerUser": 1, 
+                                                "maxApplications": 47, 
+                                                "maxApplicationsPerUser": 47, 
+                                                "maxCapacity": 100.0, 
+                                                "numActiveApplications": 0, 
+                                                "numApplications": 0, 
+                                                "numContainers": 0, 
+                                                "numPendingApplications": 0, 
+                                                "queueName": "a1b", 
+                                                "resourcesUsed": {
+                                                    "memory": 0, 
+                                                    "vCores": 0
+                                                }, 
+                                                "state": "RUNNING", 
+                                                "type": "capacitySchedulerLeafQueueInfo", 
+                                                "usedCapacity": 0.0, 
+                                                "usedResources": "<memory:0, vCores:0>", 
+                                                "userLimit": 100, 
+                                                "userLimitFactor": 1.0, 
+                                                "users": null
+                                            }
+                                        ]
+                                    }, 
+                                    "resourcesUsed": {
+                                        "memory": 0, 
+                                        "vCores": 0
+                                    }, 
+                                    "state": "RUNNING", 
+                                    "usedCapacity": 0.0, 
+                                    "usedResources": "<memory:0, vCores:0>"
+                                }, 
+                                {
+                                    "absoluteCapacity": 7.35, 
+                                    "absoluteMaxCapacity": 50.0, 
+                                    "absoluteUsedCapacity": 0.0, 
+                                    "capacity": 70.0, 
+                                    "maxActiveApplications": 1, 
+                                    "maxActiveApplicationsPerUser": 100, 
+                                    "maxApplications": 735, 
+                                    "maxApplicationsPerUser": 73500, 
+                                    "maxCapacity": 100.0, 
+                                    "numActiveApplications": 0, 
+                                    "numApplications": 0, 
+                                    "numContainers": 0, 
+                                    "numPendingApplications": 0, 
+                                    "queueName": "a2", 
+                                    "resourcesUsed": {
+                                        "memory": 0, 
+                                        "vCores": 0
+                                    }, 
+                                    "state": "RUNNING", 
+                                    "type": "capacitySchedulerLeafQueueInfo", 
+                                    "usedCapacity": 0.0, 
+                                    "usedResources": "<memory:0, vCores:0>", 
+                                    "userLimit": 100, 
+                                    "userLimitFactor": 100.0, 
+                                    "users": null
+                                }
+                            ]
+                        }, 
+                        "resourcesUsed": {
+                            "memory": 0, 
+                            "vCores": 0
+                        }, 
+                        "state": "RUNNING", 
+                        "usedCapacity": 0.0, 
+                        "usedResources": "<memory:0, vCores:0>"
+                    }, 
+                    {
+                        "absoluteCapacity": 89.5, 
+                        "absoluteMaxCapacity": 100.0, 
+                        "absoluteUsedCapacity": 0.0, 
+                        "capacity": 89.5, 
+                        "maxCapacity": 100.0, 
+                        "numApplications": 2, 
+                        "queueName": "b", 
+                        "queues": {
+                            "queue": [
+                                {
+                                    "absoluteCapacity": 53.7, 
+                                    "absoluteMaxCapacity": 100.0, 
+                                    "absoluteUsedCapacity": 0.0, 
+                                    "capacity": 60.000004, 
+                                    "maxActiveApplications": 1, 
+                                    "maxActiveApplicationsPerUser": 100, 
+                                    "maxApplications": 5370, 
+                                    "maxApplicationsPerUser": 537000, 
+                                    "maxCapacity": 100.0, 
+                                    "numActiveApplications": 1, 
+                                    "numApplications": 2, 
+                                    "numContainers": 0, 
+                                    "numPendingApplications": 1, 
+                                    "queueName": "b1", 
+                                    "resourcesUsed": {
+                                        "memory": 0, 
+                                        "vCores": 0
+                                    }, 
+                                    "state": "RUNNING", 
+                                    "type": "capacitySchedulerLeafQueueInfo", 
+                                    "usedCapacity": 0.0, 
+                                    "usedResources": "<memory:0, vCores:0>", 
+                                    "userLimit": 100, 
+                                    "userLimitFactor": 100.0, 
+                                    "users": {
+                                        "user": [
+                                            {
+                                                "numActiveApplications": 0, 
+                                                "numPendingApplications": 1, 
+                                                "resourcesUsed": {
+                                                    "memory": 0, 
+                                                    "vCores": 0
+                                                }, 
+                                                "username": "user2"
+                                            }, 
+                                            {
+                                                "numActiveApplications": 1, 
+                                                "numPendingApplications": 0, 
+                                                "resourcesUsed": {
+                                                    "memory": 0, 
+                                                    "vCores": 0
+                                                }, 
+                                                "username": "user1"
+                                            }
+                                        ]
+                                    }
+                                }, 
+                                {
+                                    "absoluteCapacity": 35.3525, 
+                                    "absoluteMaxCapacity": 100.0, 
+                                    "absoluteUsedCapacity": 0.0, 
+                                    "capacity": 39.5, 
+                                    "maxActiveApplications": 1, 
+                                    "maxActiveApplicationsPerUser": 100, 
+                                    "maxApplications": 3535, 
+                                    "maxApplicationsPerUser": 353500, 
+                                    "maxCapacity": 100.0, 
+                                    "numActiveApplications": 0, 
+                                    "numApplications": 0, 
+                                    "numContainers": 0, 
+                                    "numPendingApplications": 0, 
+                                    "queueName": "b2", 
+                                    "resourcesUsed": {
+                                        "memory": 0, 
+                                        "vCores": 0
+                                    }, 
+                                    "state": "RUNNING", 
+                                    "type": "capacitySchedulerLeafQueueInfo", 
+                                    "usedCapacity": 0.0, 
+                                    "usedResources": "<memory:0, vCores:0>", 
+                                    "userLimit": 100, 
+                                    "userLimitFactor": 100.0, 
+                                    "users": null
+                                }, 
+                                {
+                                    "absoluteCapacity": 0.4475, 
+                                    "absoluteMaxCapacity": 100.0, 
+                                    "absoluteUsedCapacity": 0.0, 
+                                    "capacity": 0.5, 
+                                    "maxActiveApplications": 1, 
+                                    "maxActiveApplicationsPerUser": 100, 
+                                    "maxApplications": 44, 
+                                    "maxApplicationsPerUser": 4400, 
+                                    "maxCapacity": 100.0, 
+                                    "numActiveApplications": 0, 
+                                    "numApplications": 0, 
+                                    "numContainers": 0, 
+                                    "numPendingApplications": 0, 
+                                    "queueName": "b3", 
+                                    "resourcesUsed": {
+                                        "memory": 0, 
+                                        "vCores": 0
+                                    }, 
+                                    "state": "RUNNING", 
+                                    "type": "capacitySchedulerLeafQueueInfo", 
+                                    "usedCapacity": 0.0, 
+                                    "usedResources": "<memory:0, vCores:0>", 
+                                    "userLimit": 100, 
+                                    "userLimitFactor": 100.0, 
+                                    "users": null
+                                }
+                            ]
+                        }, 
+                        "resourcesUsed": {
+                            "memory": 0, 
+                            "vCores": 0
+                        }, 
+                        "state": "RUNNING", 
+                        "usedCapacity": 0.0, 
+                        "usedResources": "<memory:0, vCores:0>"
+                    }
+                ]
+            }, 
+            "type": "capacityScheduler", 
+            "usedCapacity": 0.0
+        }
+    }
+}
+```json
+
+**XML response**
+
+HTTP Request:
+
+      Accept: application/xml
+      GET http://<rm http address:port>/ws/v1/cluster/scheduler
+
+Response Header:
+
+      HTTP/1.1 200 OK
+      Content-Type: application/xml
+      Content-Length: 5778
+      Server: Jetty(6.1.26)
+
+Response Body:
+
+```xml
+<?xml version="1.0" encoding="UTF-8" standalone="yes"?>
+<scheduler>
+  <schedulerInfo xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:type="capacityScheduler">
+    <capacity>100.0</capacity>
+    <usedCapacity>0.0</usedCapacity>
+    <maxCapacity>100.0</maxCapacity>
+    <queueName>root</queueName>
+    <queues>
+      <queue>
+        <capacity>10.5</capacity>
+        <usedCapacity>0.0</usedCapacity>
+        <maxCapacity>50.0</maxCapacity>
+        <absoluteCapacity>10.5</absoluteCapacity>
+        <absoluteMaxCapacity>50.0</absoluteMaxCapacity>
+        <absoluteUsedCapacity>0.0</absoluteUsedCapacity>
+        <numApplications>0</numApplications>
+        <usedResources>&lt;memory:0, vCores:0&gt;</usedResources>
+        <queueName>a</queueName>
+        <state>RUNNING</state>
+        <queues>
+          <queue>
+            <capacity>30.000002</capacity>
+            <usedCapacity>0.0</usedCapacity>
+            <maxCapacity>50.0</maxCapacity>
+            <absoluteCapacity>3.15</absoluteCapacity>
+            <absoluteMaxCapacity>25.0</absoluteMaxCapacity>
+            <absoluteUsedCapacity>0.0</absoluteUsedCapacity>
+            <numApplications>0</numApplications>
+            <usedResources>&lt;memory:0, vCores:0&gt;</usedResources>
+            <queueName>a1</queueName>
+            <state>RUNNING</state>
+            <queues>
+              <queue xsi:type="capacitySchedulerLeafQueueInfo">
+                <capacity>85.0</capacity>
+                <usedCapacity>0.0</usedCapacity>
+                <maxCapacity>100.0</maxCapacity>
+                <absoluteCapacity>2.6775</absoluteCapacity>
+                <absoluteMaxCapacity>25.0</absoluteMaxCapacity>
+                <absoluteUsedCapacity>0.0</absoluteUsedCapacity>
+                <numApplications>0</numApplications>
+                <usedResources>&lt;memory:0, vCores:0&gt;</usedResources>
+                <queueName>a1a</queueName>
+                <state>RUNNING</state>
+                <resourcesUsed>
+                  <memory>0</memory>
+                  <vCores>0</vCores>
+                </resourcesUsed>
+                <numActiveApplications>0</numActiveApplications>
+                <numPendingApplications>0</numPendingApplications>
+                <numContainers>0</numContainers>
+                <maxApplications>267</maxApplications>
+                <maxApplicationsPerUser>267</maxApplicationsPerUser>
+                <maxActiveApplications>1</maxActiveApplications>
+                <maxActiveApplicationsPerUser>1</maxActiveApplicationsPerUser>
+                <userLimit>100</userLimit>
+                <users/>
+                <userLimitFactor>1.0</userLimitFactor>
+              </queue>
+              <queue xsi:type="capacitySchedulerLeafQueueInfo">
+                <capacity>15.000001</capacity>
+                <usedCapacity>0.0</usedCapacity>
+                <maxCapacity>100.0</maxCapacity>
+                <absoluteCapacity>0.47250003</absoluteCapacity>
+                <absoluteMaxCapacity>25.0</absoluteMaxCapacity>
+                <absoluteUsedCapacity>0.0</absoluteUsedCapacity>
+                <numApplications>0</numApplications>
+                <usedResources>&lt;memory:0, vCores:0&gt;</usedResources>
+                <queueName>a1b</queueName>
+                <state>RUNNING</state>
+                <resourcesUsed>
+                  <memory>0</memory>
+                  <vCores>0</vCores>
+                </resourcesUsed>
+                <numActiveApplications>0</numActiveApplications>
+                <numPendingApplications>0</numPendingApplications>
+                <numContainers>0</numContainers>
+                <maxApplications>47</maxApplications>
+                <maxApplicationsPerUser>47</maxApplicationsPerUser>
+                <maxActiveApplications>1</maxActiveApplications>
+                <maxActiveApplicationsPerUser>1</maxActiveApplicationsPerUser>
+                <userLimit>100</userLimit>
+                <users/>
+                <userLimitFactor>1.0</userLimitFactor>
+              </queue>
+            </queues>
+            <resourcesUsed>
+              <memory>0</memory>
+              <vCores>0</vCores>
+            </resourcesUsed>
+          </queue>
+          <queue xsi:type="capacitySchedulerLeafQueueInfo">
+            <capacity>70.0</capacity>
+            <usedCapacity>0.0</usedCapacity>
+            <maxCapacity>100.0</maxCapacity>
+            <absoluteCapacity>7.35</absoluteCapacity>
+            <absoluteMaxCapacity>50.0</absoluteMaxCapacity>
+            <absoluteUsedCapacity>0.0</absoluteUsedCapacity>
+            <numApplications>0</numApplications>
+            <usedResources>&lt;memory:0, vCores:0&gt;</usedResources>
+            <queueName>a2</queueName>
+            <state>RUNNING</state>
+            <resourcesUsed>
+              <memory>0</memory>
+              <vCores>0</vCores>
+            </resourcesUsed>
+            <numActiveApplications>0</numActiveApplications>
+            <numPendingApplications>0</numPendingApplications>
+            <numContainers>0</numContainers>
+            <maxApplications>735</maxApplications>
+            <maxApplicationsPerUser>73500</maxApplicationsPerUser>
+            <maxActiveApplications>1</maxActiveApplications>
+            <maxActiveApplicationsPerUser>100</maxActiveApplicationsPerUser>
+            <userLimit>100</userLimit>
+            <users/>
+            <userLimitFactor>100.0</userLimitFactor>
+          </queue>
+        </queues>
+        <resourcesUsed>
+          <memory>0</memory>
+          <vCores>0</vCores>
+        </resourcesUsed>
+      </queue>
+      <queue>
+        <capacity>89.5</capacity>
+        <usedCapacity>0.0</usedCapacity>
+        <maxCapacity>100.0</maxCapacity>
+        <absoluteCapacity>89.5</absoluteCapacity>
+        <absoluteMaxCapacity>100.0</absoluteMaxCapacity>
+        <absoluteUsedCapacity>0.0</absoluteUsedCapacity>
+        <numApplications>2</numApplications>
+        <usedResources>&lt;memory:0, vCores:0&gt;</usedResources>
+        <queueName>b</queueName>
+        <state>RUNNING</state>
+        <queues>
+          <queue xsi:type="capacitySchedulerLeafQueueInfo">
+            <capacity>60.000004</capacity>
+            <usedCapacity>0.0</usedCapacity>
+            <maxCapacity>100.0</maxCapacity>
+            <absoluteCapacity>53.7</absoluteCapacity>
+            <absoluteMaxCapacity>100.0</absoluteMaxCapacity>
+            <absoluteUsedCapacity>0.0</absoluteUsedCapacity>
+            <numApplications>2</numApplications>
+            <usedResources>&lt;memory:0, vCores:0&gt;</usedResources>
+            <queueName>b1</queueName>
+            <state>RUNNING</state>
+            <resourcesUsed>
+              <memory>0</memory>
+              <vCores>0</vCores>
+            </resourcesUsed>
+            <numActiveApplications>1</numActiveApplications>
+            <numPendingApplications>1</numPendingApplications>
+            <numContainers>0</numContainers>
+            <maxApplications>5370</maxApplications>
+            <maxApplicationsPerUser>537000</maxApplicationsPerUser>
+            <maxActiveApplications>1</maxActiveApplications>
+            <maxActiveApplicationsPerUser>100</maxActiveApplicationsPerUser>
+            <userLimit>100</userLimit>
+            <users>
+              <user>
+                <username>user2</username>
+                <resourcesUsed>
+                  <memory>0</memory>
+                  <vCores>0</vCores>
+                </resourcesUsed>
+                <numPendingApplications>1</numPendingApplications>
+                <numActiveApplications>0</numActiveApplications>
+              </user>
+              <user>
+                <username>user1</username>
+                <resourcesUsed>
+                  <memory>0</memory>
+                  <vCores>0</vCores>
+                </resourcesUsed>
+                <numPendingApplications>0</numPendingApplications>
+                <numActiveApplications>1</numActiveApplications>
+              </user>
+            </users>
+            <userLimitFactor>100.0</userLimitFactor>
+          </queue>
+          <queue xsi:type="capacitySchedulerLeafQueueInfo">
+            <capacity>39.5</capacity>
+            <usedCapacity>0.0</usedCapacity>
+            <maxCapacity>100.0</maxCapacity>
+            <absoluteCapacity>35.3525</absoluteCapacity>
+            <absoluteMaxCapacity>100.0</absoluteMaxCapacity>
+            <absoluteUsedCapacity>0.0</absoluteUsedCapacity>
+            <numApplications>0</numApplications>
+            <usedResources>&lt;memory:0, vCores:0&gt;</usedResources>
+            <queueName>b2</queueName>
+            <state>RUNNING</state>
+            <resourcesUsed>
+              <memory>0</memory>
+              <vCores>0</vCores>
+            </resourcesUsed>
+            <numActiveApplications>0</numActiveApplications>
+            <numPendingApplications>0</numPendingApplications>
+            <numContainers>0</numContainers>
+            <maxApplications>3535</maxApplications>
+            <maxApplicationsPerUser>353500</maxApplicationsPerUser>
+            <maxActiveApplications>1</maxActiveApplications>
+            <maxActiveApplicationsPerUser>100</maxActiveApplicationsPerUser>
+            <userLimit>100</userLimit>
+            <users/>
+            <userLimitFactor>100.0</userLimitFactor>
+          </queue>
+          <queue xsi:type="capacitySchedulerLeafQueueInfo">
+            <capacity>0.5</capacity>
+            <usedCapacity>0.0</usedCapacity>
+            <maxCapacity>100.0</maxCapacity>
+            <absoluteCapacity>0.4475</absoluteCapacity>
+            <absoluteMaxCapacity>100.0</absoluteMaxCapacity>
+            <absoluteUsedCapacity>0.0</absoluteUsedCapacity>
+            <numApplications>0</numApplications>
+            <usedResources>&lt;memory:0, vCores:0&gt;</usedResources>
+            <queueName>b3</queueName>
+            <state>RUNNING</state>
+            <resourcesUsed>
+              <memory>0</memory>
+              <vCores>0</vCores>
+            </resourcesUsed>
+            <numActiveApplications>0</numActiveApplications>
+            <numPendingApplications>0</numPendingApplications>
+            <numContainers>0</numContainers>
+            <maxApplications>44</maxApplications>
+            <maxApplicationsPerUser>4400</maxApplicationsPerUser>
+            <maxActiveApplications>1</maxActiveApplications>
+            <maxActiveApplicationsPerUser>100</maxActiveApplicationsPerUser>
+            <userLimit>100</userLimit>
+            <users/>
+            <userLimitFactor>100.0</userLimitFactor>
+          </queue>
+        </queues>
+        <resourcesUsed>
+          <memory>0</memory>
+          <vCores>0</vCores>
+        </resourcesUsed>
+      </queue>
+    </queues>
+  </schedulerInfo>
+</scheduler>
+```
+
+### Fifo Scheduler API
+
+### Elements of the *schedulerInfo* object
+
+| Item | Data Type | Description |
+|:---- |:---- |:---- |
+| type | string | Scheduler type - fifoScheduler |
+| capacity | float | Queue capacity in percentage |
+| usedCapacity | float | Used queue capacity in percentage |
+| qstate | string | State of the queue - valid values are: STOPPED, RUNNING |
+| minQueueMemoryCapacity | int | Minimum queue memory capacity |
+| maxQueueMemoryCapacity | int | Maximum queue memory capacity |
+| numNodes | int | The total number of nodes |
+| usedNodeCapacity | int | The used node capacity |
+| availNodeCapacity | int | The available node capacity |
+| totalNodeCapacity | int | The total node capacity |
+| numContainers | int | The number of containers |
+
+#### Response Examples
+
+**JSON response**
+
+HTTP Request:
+
+      GET http://<rm http address:port>/ws/v1/cluster/scheduler
+
+Response Header:
+
+      HTTP/1.1 200 OK
+      Content-Type: application/json
+      Transfer-Encoding: chunked
+      Server: Jetty(6.1.26)
+
+Response Body:
+
+```json
+{
+  "scheduler":
+  {
+    "schedulerInfo":
+    {
+      "type":"fifoScheduler",
+      "capacity":1,
+      "usedCapacity":"NaN",
+      "qstate":"RUNNING",
+      "minQueueMemoryCapacity":1024,
+      "maxQueueMemoryCapacity":10240,
+      "numNodes":0,
+      "usedNodeCapacity":0,
+      "availNodeCapacity":0,
+      "totalNodeCapacity":0,
+      "numContainers":0
+    }
+  }
+}
+```
+
+**XML response**
+
+HTTP Request:
+
+      GET http://<rm http address:port>/ws/v1/cluster/scheduler
+      Accept: application/xml
+
+Response Header:
+
+      HTTP/1.1 200 OK
+      Content-Type: application/xml
+      Content-Length: 432
+      Server: Jetty(6.1.26)
+
+Response Body:
+
+```xml
+<?xml version="1.0" encoding="UTF-8" standalone="yes"?>
+<scheduler>
+  <schedulerInfo xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:type="fifoScheduler">
+    <capacity>1.0</capacity>
+    <usedCapacity>NaN</usedCapacity>
+    <qstate>RUNNING</qstate>
+    <minQueueMemoryCapacity>1024</minQueueMemoryCapacity>
+    <maxQueueMemoryCapacity>10240</maxQueueMemoryCapacity>
+    <numNodes>0</numNodes>
+    <usedNodeCapacity>0</usedNodeCapacity>
+    <availNodeCapacity>0</availNodeCapacity>
+    <totalNodeCapacity>0</totalNodeCapacity>
+    <numContainers>0</numContainers>
+  </schedulerInfo>
+</scheduler>
+```
+
+Cluster Applications API
+------------------------
+
+With the Applications API, you can obtain a collection of resources, each of which represents an application. When you run a GET operation on this resource, you obtain a collection of Application Objects.
+
+### URI
+
+      * http://<rm http address:port>/ws/v1/cluster/apps
+
+### HTTP Operations Supported
+
+      * GET
+
+### Query Parameters Supported
+
+Multiple parameters can be specified for GET operations. The started and finished times have a begin and end parameter to allow you to specify ranges. For example, one could request all applications that started between 1:00am and 2:00pm on 12/19/2011 with startedTimeBegin=1324256400&startedTimeEnd=1324303200. If the Begin parameter is not specified, it defaults to 0, and if the End parameter is not specified, it defaults to infinity.
+
+      * state [deprecated] - state of the application
+      * states - applications matching the given application states, specified as a comma-separated list.
+      * finalStatus - the final status of the application - reported by the application itself
+      * user - user name
+      * queue - queue name
+      * limit - total number of app objects to be returned
+      * startedTimeBegin - applications with start time beginning with this time, specified in ms since epoch
+      * startedTimeEnd - applications with start time ending with this time, specified in ms since epoch
+      * finishedTimeBegin - applications with finish time beginning with this time, specified in ms since epoch
+      * finishedTimeEnd - applications with finish time ending with this time, specified in ms since epoch
+      * applicationTypes - applications matching the given application types, specified as a comma-separated list.
+      * applicationTags - applications matching any of the given application tags, specified as a comma-separated list.
+
+### Elements of the *apps* (Applications) object
+
+When you make a request for the list of applications, the information will be returned as a collection of app objects. See also [Application API](#Application_API) for syntax of the app object.
+
+| Item | Data Type | Description |
+|:---- |:---- |:---- |
+| app | array of app objects(JSON)/zero or more application objects(XML) | The collection of application objects |
+
+### Response Examples
+
+**JSON response**
+
+HTTP Request:
+
+      GET http://<rm http address:port>/ws/v1/cluster/apps
+
+Response Header:
+
+      HTTP/1.1 200 OK
+      Content-Type: application/json
+      Transfer-Encoding: chunked
+      Server: Jetty(6.1.26)
+
+Response Body:
+
+```json
+{
+  "apps":
+  {
+    "app":
+    [
+       {
+          "finishedTime" : 1326815598530,
+          "amContainerLogs" : "http://host.domain.com:8042/node/containerlogs/container_1326815542473_0001_01_000001",
+          "trackingUI" : "History",
+          "state" : "FINISHED",
+          "user" : "user1",
+          "id" : "application_1326815542473_0001",
+          "clusterId" : 1326815542473,
+          "finalStatus" : "SUCCEEDED",
+          "amHostHttpAddress" : "host.domain.com:8042",
+          "progress" : 100,
+          "name" : "word count",
+          "startedTime" : 1326815573334,
+          "elapsedTime" : 25196,
+          "diagnostics" : "",
+          "trackingUrl" : "http://host.domain.com:8088/proxy/application_1326815542473_0001/jobhistory/job/job_1326815542473_1_1",
+          "queue" : "default",
+          "allocatedMB" : 0,
+          "allocatedVCores" : 0,
+          "runningContainers" : 0,
+          "memorySeconds" : 151730,
+          "vcoreSeconds" : 103
+       },
+       {
+          "finishedTime" : 1326815789546,
+          "amContainerLogs" : "http://host.domain.com:8042/node/containerlogs/container_1326815542473_0002_01_000001",
+          "trackingUI" : "History",
+          "state" : "FINISHED",
+          "user" : "user1",
+          "id" : "application_1326815542473_0002",
+          "clusterId" : 1326815542473,
+          "finalStatus" : "SUCCEEDED",
+          "amHostHttpAddress" : "host.domain.com:8042",
+          "progress" : 100,
+          "name" : "Sleep job",
+          "startedTime" : 1326815641380,
+          "elapsedTime" : 148166,
+          "diagnostics" : "",
+          "trackingUrl" : "http://host.domain.com:8088/proxy/application_1326815542473_0002/jobhistory/job/job_1326815542473_2_2",
+          "queue" : "default",
+          "allocatedMB" : 0,
+          "allocatedVCores" : 0,
+          "runningContainers" : 1,
+          "memorySeconds" : 640064,
+          "vcoreSeconds" : 442
+       } 
+    ]
+  }
+}
+```
+
+**XML response**
+
+HTTP Request:
+
+      GET http://<rm http address:port>/ws/v1/cluster/apps
+      Accept: application/xml
+
+Response Header:
+
+      HTTP/1.1 200 OK
+      Content-Type: application/xml
+      Content-Length: 2459
+      Server: Jetty(6.1.26)
+
+Response Body:
+
+```xml
+<?xml version="1.0" encoding="UTF-8" standalone="yes"?>
+<apps>
+  <app>
+    <id>application_1326815542473_0001</id>
+    <user>user1</user>
+    <name>word count</name>
+    <applicationType>MAPREDUCE</applicationType>
+    <queue>default</queue>
+    <state>FINISHED</state>
+    <finalStatus>SUCCEEDED</finalStatus>
+    <progress>100.0</progress>
+    <trackingUI>History</trackingUI>
+    <trackingUrl>http://host.domain.com:8088/proxy/application_1326815542473_0001/jobhistory/job/job_1326815542473_1_1</trackingUrl>
+    <diagnostics/>
+    <clusterId>1326815542473</clusterId>
+    <startedTime>1326815573334</startedTime>
+    <finishedTime>1326815598530</finishedTime>
+    <elapsedTime>25196</elapsedTime>
+    <amContainerLogs>http://host.domain.com:8042/node/containerlogs/container_1326815542473_0001_01_000001</amContainerLogs>
+    <amHostHttpAddress>host.domain.com:8042</amHostHttpAddress>
+    <allocatedMB>0</allocatedMB>
+    <allocatedVCores>0</allocatedVCores>
+    <runningContainers>0</runningContainers>
+    <memorySeconds>151730</memorySeconds>
+    <vcoreSeconds>103</vcoreSeconds>
+  </app>
+  <app>
+    <id>application_1326815542473_0002</id>
+    <user>user1</user>
+    <name>Sleep job</name>
+    <applicationType>YARN</applicationType>
+    <queue>default</queue>
+    <state>FINISHED</state>
+    <finalStatus>SUCCEEDED</finalStatus>
+    <progress>100.0</progress>
+    <trackingUI>History</trackingUI>
+    <trackingUrl>http://host.domain.com:8088/proxy/application_1326815542473_0002/jobhistory/job/job_1326815542473_2_2</trackingUrl>
+    <diagnostics/>
+    <clusterId>1326815542473</clusterId>
+    <startedTime>1326815641380</startedTime>
+    <finishedTime>1326815789546</finishedTime>
+    <elapsedTime>148166</elapsedTime>
+    <amContainerLogs>http://host.domain.com:8042/node/containerlogs/container_1326815542473_0002_01_000001</amContainerLogs>
+    <amHostHttpAddress>host.domain.com:8042</amHostHttpAddress>
+    <allocatedMB>0</allocatedMB>
+    <allocatedVCores>0</allocatedVCores>
+    <runningContainers>0</runningContainers>
+    <memorySeconds>640064</memorySeconds>
+    <vcoreSeconds>442</vcoreSeconds>
+  </app>
+</apps>
+```
+
+Cluster Application Statistics API
+----------------------------------
+
+With the Application Statistics API, you can obtain a collection of triples, each of which contains the application type, the application state and the number of applications of this type and this state in ResourceManager context. Note that with the performance concern, we currently only support at most one applicationType per query. We may support multiple applicationTypes per query as well as more statistics in the future. When you run a GET operation on this resource, you obtain a collection of statItem objects.
+
+### URI
+
+      * http://<rm http address:port>/ws/v1/cluster/appstatistics
+
+### HTTP Operations Supported
+
+      * GET
+
+### Query Parameters Required
+
+Two paramters can be specified. The parameters are case insensitive.
+
+      * states - states of the applications, specified as a comma-separated list. If states is not provided, the API will enumerate all application states and return the counts of them.
+      * applicationTypes - types of the applications, specified as a comma-separated list. If applicationTypes is not provided, the API will count the applications of any application type. In this case, the response shows * to indicate any application type. Note that we only support at most one applicationType temporarily. Otherwise, users will expect an BadRequestException.
+
+### Elements of the *appStatInfo* (statItems) object
+
+When you make a request for the list of statistics items, the information will be returned as a collection of statItem objects
+
+| Item | Data Type | Description |
+|:---- |:---- |:---- |
+| statItem | array of statItem objects(JSON)/zero or more statItem objects(XML) | The collection of statItem objects |
+
+### Response Examples
+
+**JSON response**
+
+HTTP Request:
+
+      GET http://<rm http address:port>/ws/v1/cluster/appstatistics?states=accepted,running,finished&applicationTypes=mapreduce
+
+Response Header:
+
+      HTTP/1.1 200 OK
+      Content-Type: application/json
+      Transfer-Encoding: chunked
+      Server: Jetty(6.1.26)
+
+Response Body:
+
+```json
+{
+  "appStatInfo":
+  {
+    "statItem":
+    [
+       {
+          "state" : "accepted",
+          "type" : "mapreduce",
+          "count" : 4
+       },
+       {
+          "state" : "running",
+          "type" : "mapreduce",
+          "count" : 1
+       },
+       {
+          "state" : "finished",
+          "type" : "mapreduce",
+          "count" : 7
+       }
+    ]
+  }
+}
+```
+
+**XML response**
+
+HTTP Request:
+
+      GET http://<rm http address:port>/ws/v1/cluster/appstatistics?states=accepted,running,finished&applicationTypes=mapreduce
+      Accept: application/xml
+
+Response Header:
+
+      HTTP/1.1 200 OK
+      Content-Type: application/xml
+      Content-Length: 2459
+      Server: Jetty(6.1.26)
+
+Response Body:
+
+```xml
+<?xml version="1.0" encoding="UTF-8" standalone="yes"?>
+<appStatInfo>
+  <statItem>
+    <state>accepted</state>
+    <type>mapreduce</type>
+    <count>4</count>
+  </statItem>
+  <statItem>
+    <state>running</state>
+    <type>mapreduce</type>
+    <count>1</count>
+  </statItem>
+  <statItem>
+    <state>finished</state>
+    <type>mapreduce</type>
+    <count>7</count>
+  </statItem>
+</appStatInfo>
+```
+
+Cluster Application API
+-----------------------
+
+An application resource contains information about a particular application that was submitted to a cluster.
+
+### URI
+
+Use the following URI to obtain an app object, from a application identified by the appid value.
+
+      * http://<rm http address:port>/ws/v1/cluster/apps/{appid}
+
+### HTTP Operations Supported
+
+      * GET
+
+### Query Parameters Supported
+
+      None
+
+### Elements of the *app* (Application) object
+
+Note that depending on security settings a user might not be able to see all the fields.
+
+| Item | Data Type | Description |
+|:---- |:---- |:---- |
+| id | string | The application id |
+| user | string | The user who started the application |
+| name | string | The application name |
+| Application Type | string | The application type |
+| queue | string | The queue the application was submitted to |
+| state | string | The application state according to the ResourceManager - valid values are members of the YarnApplicationState enum: NEW, NEW\_SAVING, SUBMITTED, ACCEPTED, RUNNING, FINISHED, FAILED, KILLED |
+| finalStatus | string | The final status of the application if finished - reported by the application itself - valid values are: UNDEFINED, SUCCEEDED, FAILED, KILLED |
+| progress | float | The progress of the application as a percent |
+| trackingUI | string | Where the tracking url is currently pointing - History (for history server) or ApplicationMaster |
+| trackingUrl | string | The web URL that can be used to track the application |
+| diagnostics | string | Detailed diagnostics information |
+| clusterId | long | The cluster id |
+| startedTime | long | The time in which application started (in ms since epoch) |
+| finishedTime | long | The time in which the application finished (in ms since epoch) |
+| elapsedTime | long | The elapsed time since the application started (in ms) |
+| amContainerLogs | string | The URL of the application master container logs |
+| amHostHttpAddress | string | The nodes http address of the application master |
+| allocatedMB | int | The sum of memory in MB allocated to the application's running containers |
+| allocatedVCores | int | The sum of virtual cores allocated to the application's running containers |
+| runningContainers | int | The number of containers currently running for the application |
+| memorySeconds | long | The amount of memory the application has allocated (megabyte-seconds) |
+| vcoreSeconds | long | The amount of CPU resources the application has allocated (virtual core-seconds) |
+
+### Response Examples
+
+**JSON response**
+
+HTTP Request:
+
+      GET http://<rm http address:port>/ws/v1/cluster/apps/application_1326821518301_0005
+
+Response Header:
+
+      HTTP/1.1 200 OK
+      Content-Type: application/json
+      Transfer-Encoding: chunked
+      Server: Jetty(6.1.26)
+
+Response Body:
+
+```json
+{
+   "app" : {
+      "finishedTime" : 1326824991300,
+      "amContainerLogs" : "http://host.domain.com:8042/node/containerlogs/container_1326821518301_0005_01_000001",
+      "trackingUI" : "History",
+      "state" : "FINISHED",
+      "user" : "user1",
+      "id" : "application_1326821518301_0005",
+      "clusterId" : 1326821518301,
+      "finalStatus" : "SUCCEEDED",
+      "amHostHttpAddress" : "host.domain.com:8042",
+      "progress" : 100,
+      "name" : "Sleep job",
+      "applicationType" : "Yarn",
+      "startedTime" : 1326824544552,
+      "elapsedTime" : 446748,
+      "diagnostics" : "",
+      "trackingUrl" : "http://host.domain.com:8088/proxy/application_1326821518301_0005/jobhistory/job/job_1326821518301_5_5",
+      "queue" : "a1",
+      "memorySeconds" : 151730,
+      "vcoreSeconds" : 103
+   }
+}
+```
+
+**XML response**
+
+HTTP Request:
+
+      GET http://<rm http address:port>/ws/v1/cluster/apps/application_1326821518301_0005
+      Accept: application/xml
+
+Response Header:
+
+      HTTP/1.1 200 OK
+      Content-Type: application/xml
+      Content-Length: 847
+      Server: Jetty(6.1.26)
+
+Response Body:
+
+```xml
+<?xml version="1.0" encoding="UTF-8" standalone="yes"?>
+<app>
+  <id>application_1326821518301_0005</id>
+  <user>user1</user>
+  <name>Sleep job</name>
+  <queue>a1</queue>
+  <state>FINISHED</state>
+  <finalStatus>SUCCEEDED</finalStatus>
+  <progress>100.0</progress>
+  <trackingUI>History</trackingUI>
+  <trackingUrl>http://host.domain.com:8088/proxy/application_1326821518301_0005/jobhistory/job/job_1326821518301_5_5</trackingUrl>
+  <diagnostics/>
+  <clusterId>1326821518301</clusterId>
+  <startedTime>1326824544552</startedTime>
+  <finishedTime>1326824991300</finishedTime>
+  <elapsedTime>446748</elapsedTime>
+  <amContainerLogs>http://host.domain.com:8042/node/containerlogs/container_1326821518301_0005_01_000001</amContainerLogs>
+  <amHostHttpAddress>host.domain.com:8042</amHostHttpAddress>
+  <memorySeconds>151730</memorySeconds>
+  <vcoreSeconds>103</vcoreSeconds>
+</app>
+```
+
+Cluster Application Attempts API
+--------------------------------
+
+With the application attempts API, you can obtain a collection of resources that represent an application attempt. When you run a GET operation on this resource, you obtain a collection of App Attempt Objects.
+
+### URI
+
+      * http://<rm http address:port>/ws/v1/cluster/apps/{appid}/appattempts
+
+### HTTP Operations Supported
+
+      * GET
+
+### Query Parameters Supported
+
+      None
+
+### Elements of the *appAttempts* object
+
+When you make a request for the list of app attempts, the information will be returned as an array of app attempt objects.
+
+appAttempts:
+
+| Item | Data Type | Description |
+|:---- |:---- |:---- |
+| appAttempt | array of app attempt objects(JSON)/zero or more app attempt objects(XML) | The collection of app attempt objects |
+
+### Elements of the *appAttempt* object
+
+| Item | Data Type | Description |
+|:---- |:---- |:---- |
+| id | string | The app attempt id |
+| nodeId | string | The node id of the node the attempt ran on |
+| nodeHttpAddress | string | The node http address of the node the attempt ran on |
+| logsLink | string | The http link to the app attempt logs |
+| containerId | string | The id of the container for the app attempt |
+| startTime | long | The start time of the attempt (in ms since epoch) |
+
+### Response Examples
+
+**JSON response**
+
+HTTP Request:
+
+      GET http://<rm http address:port>/ws/v1/cluster/apps/application_1326821518301_0005/appattempts
+
+Response Header:
+
+      HTTP/1.1 200 OK
+      Content-Type: application/json
+      Transfer-Encoding: chunked
+      Server: Jetty(6.1.26)
+
+Response Body:
+
+```json
+{
+   "appAttempts" : {
+      "appAttempt" : [
+         {
+            "nodeId" : "host.domain.com:8041",
+            "nodeHttpAddress" : "host.domain.com:8042",
+            "startTime" : 1326381444693,
+            "id" : 1,
+            "logsLink" : "http://host.domain.com:8042/node/containerlogs/container_1326821518301_0005_01_000001/user1",
+            "containerId" : "container_1326821518301_0005_01_000001"
+         }
+      ]
+   }
+}
+```
+
+**XML response**
+
+HTTP Request:
+
+      GET http://<rm http address:port>/ws/v1/cluster/apps/application_1326821518301_0005/appattempts
+      Accept: application/xml
+
+Response Header:
+
+      HTTP/1.1 200 OK
+      Content-Type: application/xml
+      Content-Length: 575
+      Server: Jetty(6.1.26)
+
+Response Body:
+
+```xml
+<?xml version="1.0" encoding="UTF-8" standalone="yes"?>
+<appAttempts>
+  <appttempt>
+    <nodeHttpAddress>host.domain.com:8042</nodeHttpAddress>
+    <nodeId>host.domain.com:8041</nodeId>
+    <id>1</id>
+    <startTime>1326381444693</startTime>
+    <containerId>container_1326821518301_0005_01_000001</containerId>
+    <logsLink>http://host.domain.com:8042/node/containerlogs/container_1326821518301_0005_01_000001/user1</logsLink>
+  </appAttempt>
+</appAttempts>
+```
+
+Cluster Nodes API
+-----------------
+
+With the Nodes API, you can obtain a collection of resources, each of which represents a node. When you run a GET operation on this resource, you obtain a collection of Node Objects.
+
+### URI
+
+      * http://<rm http address:port>/ws/v1/cluster/nodes
+
+### HTTP Operations Supported
+
+      * GET
+
+### Query Parameters Supported
+
+      * state - the state of the node
+      * healthy - true or false 
+
+### Elements of the *nodes* object
+
+When you make a request for the list of nodes, the information will be returned as a collection of node objects. See also [Node API](#Node_API) for syntax of the node object.
+
+| Item | Data Type | Description |
+|:---- |:---- |:---- |
+| node | array of node objects(JSON)/zero or more node objects(XML) | A collection of node objects |
+
+### Response Examples
+
+**JSON response**
+
+HTTP Request:
+
+      GET http://<rm http address:port>/ws/v1/cluster/nodes
+
+Response Header:
+
+      HTTP/1.1 200 OK
+      Content-Type: application/json
+      Transfer-Encoding: chunked
+      Server: Jetty(6.1.26)
+
+Response Body:
+
+```json
+{
+  "nodes":
+  {
+    "node":
+    [
+      {
+        "rack":"\/default-rack",
+        "state":"NEW",
+        "id":"h2:1235",
+        "nodeHostName":"h2",
+        "nodeHTTPAddress":"h2:2",
+        "healthStatus":"Healthy",
+        "lastHealthUpdate":1324056895432,
+        "healthReport":"Healthy",
+        "numContainers":0,
+        "usedMemoryMB":0,
+        "availMemoryMB":8192,
+        "usedVirtualCores":0,
+        "availableVirtualCores":8
+      },
+      {
+        "rack":"\/default-rack",
+        "state":"NEW",
+        "id":"h1:1234",
+        "nodeHostName":"h1",
+        "nodeHTTPAddress":"h1:2",
+        "healthStatus":"Healthy",
+        "lastHealthUpdate":1324056895092,
+        "healthReport":"Healthy",
+        "numContainers":0,
+        "usedMemoryMB":0,
+        "availMemoryMB":8192,
+        "usedVirtualCores":0,
+        "availableVirtualCores":8
+      }
+    ]
+  }
+}
+```
+
+**XML response**
+
+HTTP Request:
+
+      GET http://<rm http address:port>/ws/v1/cluster/nodes
+      Accept: application/xml
+
+Response Header:
+
+      HTTP/1.1 200 OK
+      Content-Type: application/xml
+      Content-Length: 1104
+      Server: Jetty(6.1.26)
+
+Response Body:
+
+```xml
+<?xml version="1.0" encoding="UTF-8" standalone="yes"?>
+<nodes>
+  <node>
+    <rack>/default-rack</rack>
+    <state>RUNNING</state>
+    <id>h2:1234</id>
+    <nodeHostName>h2</nodeHostName>
+    <nodeHTTPAddress>h2:2</nodeHTTPAddress>
+    <healthStatus>Healthy</healthStatus>
+    <lastHealthUpdate>1324333268447</lastHealthUpdate>
+    <healthReport>Healthy</healthReport>
+    <numContainers>0</numContainers>
+    <usedMemoryMB>0</usedMemoryMB>
+    <availMemoryMB>5120</availMemoryMB>
+    <usedVirtualCores>0</usedVirtualCores>
+    <availableVirtualCores>8</availableVirtualCores>
+  </node>
+  <node>
+    <rack>/default-rack</rack>
+    <state>RUNNING</state>
+    <id>h1:1234</id>
+    <nodeHostName>h1</nodeHostName>
+    <nodeHTTPAddress>h1:2</nodeHTTPAddress>
+    <healthStatus>Healthy</healthStatus>
+    <lastHealthUpdate>1324333268447</lastHealthUpdate>
+    <healthReport>Healthy</healthReport>
+    <numContainers>0</numContainers>
+    <usedMemoryMB>0</usedMemoryMB>
+    <availMemoryMB>5120</availMemoryMB>
+    <usedVirtualCores>0</usedVirtualCores>
+    <availableVirtualCores>8</availableVirtualCores>
+  </node>
+</nodes>
+```
+
+Cluster Node API
+----------------
+
+A node resource contains information about a node in the cluster.
+
+### URI
+
+Use the following URI to obtain a Node Object, from a node identified by the nodeid value.
+
+      * http://<rm http address:port>/ws/v1/cluster/nodes/{nodeid}
+
+### HTTP Operations Supported
+
+      * GET
+
+### Query Parameters Supported
+
+      None
+
+### Elements of the *node* object
+
+| Item | Data Type | Description |
+|:---- |:---- |:---- |
+| rack | string | The rack location of this node |
+| state | string | State of the node - valid values are: NEW, RUNNING, UNHEALTHY, DECOMMISSIONED, LOST, REBOOTED |
+| id | string | The node id |
+| nodeHostName | string | The host name of the node |
+| nodeHTTPAddress | string | The nodes HTTP address |
+| healthStatus | string | The health status of the node - Healthy or Unhealthy |
+| healthReport | string | A detailed health report |
+| lastHealthUpdate | long | The last time the node reported its health (in ms since epoch) |
+| usedMemoryMB | long | The total amount of memory currently used on the node (in MB) |
+| availMemoryMB | long | The total amount of memory currently available on the node (in MB) |
+| usedVirtualCores | long | The total number of vCores currently used on the node |
+| availableVirtualCores | long | The total number of vCores available on the node |
+| numContainers | int | The total number of containers currently running on the node |
+
+### Response Examples
+
+**JSON response**
+
+HTTP Request:
+
+      GET http://<rm http address:port>/ws/v1/cluster/nodes/h2:1235
+
+Response Header:
+
+      HTTP/1.1 200 OK
+      Content-Type: application/json
+      Transfer-Encoding: chunked
+      Server: Jetty(6.1.26)
+
+Response Body:
+
+```json
+{
+  "node":
+  {
+    "rack":"\/default-rack",
+    "state":"NEW",
+    "id":"h2:1235",
+    "nodeHostName":"h2",
+    "nodeHTTPAddress":"h2:2",
+    "healthStatus":"Healthy",
+    "lastHealthUpdate":1324056895432,
+    "healthReport":"Healthy",
+    "numContainers":0,
+    "usedMemoryMB":0,
+    "availMemoryMB":5120,
+    "usedVirtualCores":0,
+    "availableVirtualCores":8
+  }
+}
+```
+
+**XML response**
+
+HTTP Request:
+
+      GET http://<rm http address:port>/ws/v1/cluster/node/h2:1235
+      Accept: application/xml
+
+Response Header:
+
+      HTTP/1.1 200 OK
+      Content-Type: application/xml
+      Content-Length: 552
+      Server: Jetty(6.1.26)
+
+Response Body:
+
+```xml
+<?xml version="1.0" encoding="UTF-8" standalone="yes"?>
+<node>
+  <rack>/default-rack</rack>
+  <state>NEW</state>
+  <id>h2:1235</id>
+  <nodeHostName>h2</nodeHostName>
+  <nodeHTTPAddress>h2:2</nodeHTTPAddress>
+  <healthStatus>Healthy</healthStatus>
+  <lastHealthUpdate>1324333268447</lastHealthUpdate>
+  <healthReport>Healthy</healthReport>
+  <numContainers>0</numContainers>
+  <usedMemoryMB>0</usedMemoryMB>
+  <availMemoryMB>5120</availMemoryMB>
+  <usedVirtualCores>0</usedVirtualCores>
+  <availableVirtualCores>5120</availableVirtualCores>
+</node>
+```
+
+Cluster Writeable APIs
+----------------------
+
+The setions below refer to APIs which allow to create and modify applications. These APIs are currently in alpha and may change in the future.
+
+Cluster New Application API
+---------------------------
+
+With the New Application API, you can obtain an application-id which can then be used as part of the [Cluster Submit Applications API](#Cluster_Applications_APISubmit_Application) to submit applications. The response also includes the maximum resource capabilities available on the cluster.
+
+This feature is currently in the alpha stage and may change in the future.
+
+### URI
+
+      * http://<rm http address:port>/ws/v1/cluster/apps/new-application
+
+### HTTP Operations Supported
+
+      * POST
+
+### Query Parameters Supported
+
+      None
+
+### Elements of the NewApplication object
+
+The NewApplication response contains the following elements:
+
+| Item | Data Type | Description |
+|:---- |:---- |:---- |
+| application-id | string | The newly created application id |
+| maximum-resource-capabilities | object | The maximum resource capabilities available on this cluster |
+
+The *maximum-resource-capabilites* object contains the following elements:
+
+| Item | Data Type | Description |
+|:---- |:---- |:---- |
+| memory | int | The maxiumim memory available for a container |
+| vCores | int | The maximum number of cores available for a container |
+
+### Response Examples
+
+**JSON response**
+
+HTTP Request:
+
+      POST http://<rm http address:port>/ws/v1/cluster/apps/new-application
+
+Response Header:
+
+      HTTP/1.1 200 OK
+      Content-Type: application/json
+      Transfer-Encoding: chunked
+      Server: Jetty(6.1.26)
+
+Response Body:
+
+```json
+{
+  "application-id":"application_1404198295326_0003",
+  "maximum-resource-capability":
+    {
+      "memory":8192,
+      "vCores":32
+    }
+}
+```
+
+**XML response**
+
+HTTP Request:
+
+      POST http://<rm http address:port>/ws/v1/cluster/apps/new-application
+
+Response Header:
+
+      HTTP/1.1 200 OK
+      Content-Type: application/xml
+      Content-Length: 248
+      Server: Jetty(6.1.26)
+
+Response Body:
+
+```xml
+<?xml version="1.0" encoding="UTF-8" standalone="yes"?>
+<NewApplication>
+  <application-id>application_1404198295326_0003</application-id>
+  <maximum-resource-capability>
+    <memory>8192</memory>
+    <vCores>32</vCores>
+  </maximum-resource-capability>
+</NewApplication>
+```
+
+Cluster Applications API(Submit Application)
+--------------------------------------------
+
+The Submit Applications API can be used to submit applications. In case of submitting applications, you must first obtain an application-id using the [Cluster New Application API](#Cluster_New_Application_API). The application-id must be part of the request body. The response contains a URL to the application page which can be used to track the state and progress of your application.
+
+### URI
+
+      * http://<rm http address:port>/ws/v1/cluster/apps
+
+### HTTP Operations Supported
+
+      * POST
+
+### POST Response Examples
+
+POST requests can be used to submit apps to the ResourceManager. As mentioned above, an application-id must be obtained first. Successful submissions result in a 202 response code and a Location header specifying where to get information about the app. Please note that in order to submit an app, you must have an authentication filter setup for the HTTP interface. The functionality requires that a username is set in the HttpServletRequest. If no filter is setup, the response will be an "UNAUTHORIZED" response.
+
+Please note that this feature is currently in the alpha stage and may change in the future.
+
+#### Elements of the POST request object
+
+| Item | Data Type | Description |
+|:---- |:---- |:---- |
+| application-id | string | The application id |
+| application-name | string | The application name |
+| queue | string | The name of the queue to which the application should be submitted |
+| priority | int | The priority of the application |
+| am-container-spec | object | The application master container launch context, described below |
+| unmanaged-AM | boolean | Is the application using an unmanaged application master |
+| max-app-attempts | int | The max number of attempts for this application |
+| resource | object | The resources the application master requires, described below |
+| application-type | string | The application type(MapReduce, Pig, Hive, etc) |
+| keep-containers-across-application-attempts | boolean | Should YARN keep the containers used by this application instead of destroying them |
+| application-tags | object | List of application tags, please see the request examples on how to speciy the tags |
+
+Elements of the *am-container-spec* object
+
+The am-container-spec object should be used to provide the container launch context for the application master.
+
+| Item | Data Type | Description |
+|:---- |:---- |:---- |
+| local-resources | object | Object describing the resources that need to be localized, described below |
+| environment | object | Environment variables for your containers, specified as key value pairs |
+| commands | object | The commands for launching your container, in the order in which they should be executed |
+| service-data | object | Application specific service data; key is the name of the auxiliary servce, value is base-64 encoding of the data you wish to pass |
+| credentials | object | The credentials required for your application to run, described below |
+| application-acls | objec | ACLs for your application; the key can be "VIEW\_APP" or "MODIFY\_APP", the value is the list of users with the permissions |
+
+Elements of the *local-resources* object
+
+The object is a collection of key-value pairs. They key is an identifier for the resources to be localized and the value is the details of the resource. The elements of the value are described below:
+
+| Item | Data Type | Description |
+|:---- |:---- |:---- |
+| resource | string | Location of the resource to be localized |
+| type | string | Type of the resource; options are "ARCHIVE", "FILE", and "PATTERN" |
+| visibility | string | Visibility the resource to be localized; options are "PUBLIC", "PRIVATE", and "APPLICATION" |
+| size | long | Size of the resource to be localized |
+| timestamp | long | Timestamp of the resource to be localized |
+
+Elements of the *credentials* object
+
+The credentials object should be used to pass data required for the application to authenticate itself such as delegation-tokens and secrets.
+
+| Item | Data Type | Description |
+|:---- |:---- |:---- |
+| tokens | object | Tokens that you wish to pass to your application, specified as key-value pairs. The key is an identifier for the token and the value is the token(which should be obtained using the respective web-services) |
+| secrets | object | Secrets that you wish to use in your application, specified as key-value pairs. They key is an identifier and the value is the base-64 encoding of the secret |
+
+Elements of the POST request body *resource* object
+
+| Item | Data Type | Description |
+|:---- |:---- |:---- |
+| memory | int | Memory required for each container |
+| vCores | int | Virtual cores required for each container |
+
+**JSON response**
+
+HTTP Request:
+
+```json
+  POST http://<rm http address:port>/ws/v1/cluster/apps
+  Accept: application/json
+  Content-Type: application/json
+  {
+    "application-id":"application_1404203615263_0001",
+    "application-name":"test",
+    "am-container-spec":
+    {
+      "local-resources":
+      {
+        "entry":
+        [
+          {
+            "key":"AppMaster.jar",
+            "value":
+            {
+              "resource":"hdfs://hdfs-namenode:9000/user/testuser/DistributedShell/demo-app/AppMaster.jar",
+              "type":"FILE",
+              "visibility":"APPLICATION",
+              "size": 43004,
+              "timestamp": 1405452071209
+            }
+          }
+        ]
+      },
+      "commands":
+      {
+        "command":"{{JAVA_HOME}}/bin/java -Xmx10m org.apache.hadoop.yarn.applications.distributedshell.ApplicationMaster --container_memory 10 --container_vcores 1 --num_containers 1 --priority 0 1><LOG_DIR>/AppMaster.stdout 2><LOG_DIR>/AppMaster.stderr"
+      },
+      "environment":
+      {
+        "entry":
+        [
+          {
+            "key": "DISTRIBUTEDSHELLSCRIPTTIMESTAMP",
+            "value": "1405459400754"
+          },
+          {
+            "key": "CLASSPATH",
+            "value": "{{CLASSPATH}}<CPS>./*<CPS>{{HADOOP_CONF_DIR}}<CPS>{{HADOOP_COMMON_HOME}}/share/hadoop/common/*<CPS>{{HADOOP_COMMON_HOME}}/share/hadoop/common/lib/*<CPS>{{HADOOP_HDFS_HOME}}/share/hadoop/hdfs/*<CPS>{{HADOOP_HDFS_HOME}}/share/hadoop/hdfs/lib/*<CPS>{{HADOOP_YARN_HOME}}/share/hadoop/yarn/*<CPS>{{HADOOP_YARN_HOME}}/share/hadoop/yarn/lib/*<CPS>./log4j.properties"
+          },
+          {
+            "key": "DISTRIBUTEDSHELLSCRIPTLEN",
+            "value": "6"
+          },
+          {
+            "key": "DISTRIBUTEDSHELLSCRIPTLOCATION",
+            "value": "hdfs://hdfs-namenode:9000/user/testuser/demo-app/shellCommands"
+          }
+        ]
+      }
+    },
+    "unmanaged-AM":false,
+    "max-app-attempts":2,
+    "resource":
+    {
+      "memory":1024,
+      "vCores":1
+    },
+    "application-type":"YARN",
+    "keep-containers-across-application-attempts":false
+  }
+```
+
+Response Header:
+
+      HTTP/1.1 202
+      Transfer-Encoding: chunked
+      Location: http://<rm http address:port>/ws/v1/cluster/apps/application_1404203615263_0001
+      Content-Type: application/json
+      Server: Jetty(6.1.26)
+
+Response Body:
+
+      No response body
+
+**XML response**
+
+HTTP Request:
+
+```xml
+POST http://<rm http address:port>/ws/v1/cluster/apps
+Accept: application/xml
+Content-Type: application/xml
+<?xml version="1.0" encoding="UTF-8" standalone="yes"?>
+<application-submission-context>
+  <application-id>application_1404204891930_0002</application-id>
+  <application-name>test</application-name>
+  <queue>testqueue</queue>
+  <priority>3</priority>
+  <am-container-spec>
+    <local-resources>
+      <entry>
+        <key>example</key>
+        <value>
+          <resource>hdfs://hdfs-namenode:9000/user/testuser/DistributedShell/demo-app/AppMaster.jar</resource>
+          <type>FILE</type>
+          <visibility>APPLICATION</visibility>
+          <size>43004</size>
+          <timestamp>1405452071209</timestamp>
+        </value>
+      </entry>
+    </local-resources>
+    <environment>
+      <entry>
+        <key>DISTRIBUTEDSHELLSCRIPTTIMESTAMP</key>
+        <value>1405459400754</value>
+      </entry>
+      <entry>
+        <key>CLASSPATH</key>
+        <value>{{CLASSPATH}}&lt;CPS&gt;./*&lt;CPS&gt;{{HADOOP_CONF_DIR}}&lt;CPS&gt;{{HADOOP_COMMON_HOME}}/share/hadoop/common/*&lt;CPS&gt;{{HADOOP_COMMON_HOME}}/share/hadoop/common/lib/*&lt;CPS&gt;{{HADOOP_HDFS_HOME}}/share/hadoop/hdfs/*&lt;CPS&gt;{{HADOOP_HDFS_HOME}}/share/hadoop/hdfs/lib/*&lt;CPS&gt;{{HADOOP_YARN_HOME}}/share/hadoop/yarn/*&lt;CPS&gt;{{HADOOP_YARN_HOME}}/share/hadoop/yarn/lib/*&lt;CPS&gt;./log4j.properties</value>
+      </entry>
+      <entry>
+        <key>DISTRIBUTEDSHELLSCRIPTLEN</key>
+        <value>6</value>
+      </entry>
+      <entry>
+        <key>DISTRIBUTEDSHELLSCRIPTLOCATION</key>
+        <value>hdfs://hdfs-namenode:9000/user/testuser/demo-app/shellCommands</value>
+      </entry>
+    </environment>
+    <commands>
+      <command>{{JAVA_HOME}}/bin/java -Xmx10m org.apache.hadoop.yarn.applications.distributedshell.ApplicationMaster --container_memory 10 --container_vcores 1 --num_containers 1 --priority 0 1&gt;&lt;LOG_DIR&gt;/AppMaster.stdout 2&gt;&lt;LOG_DIR&gt;/AppMaster.stderr</command>
+    </commands>
+    <service-data>
+      <entry>
+        <key>test</key>
+        <value>dmFsdWUxMg</value>
+      </entry>
+    </service-data>
+    <credentials>
+      <tokens/>
+      <secrets>
+        <entry>
+          <key>secret1</key>
+          <value>c2VjcmV0MQ</value>
+        </entry>
+      </secrets>
+    </credentials>
+    <application-acls>
+      <entry>
+        <key>VIEW_APP</key>
+        <value>testuser3, testuser4</value>
+      </entry>
+      <entry>
+        <key>MODIFY_APP</key>
+        <value>testuser1, testuser2</value>
+      </entry>
+    </application-acls>
+  </am-container-spec>
+  <unmanaged-AM>false</unmanaged-AM>
+  <max-app-attempts>2</max-app-attempts>
+  <resource>
+    <memory>1024</memory>
+    <vCores>1</vCores>
+  </resource>
+  <application-type>YARN</application-type>
+  <keep-containers-across-application-attempts>false</keep-containers-across-application-attempts>
+  <application-tags>
+    <tag>tag 2</tag>
+    <tag>tag1</tag>
+  </application-tags>
+</application-submission-context>
+```
+
+Response Header:
+
+      HTTP/1.1 202
+      Transfer-Encoding: chunked
+      Location: http://<rm http address:port>/ws/v1/cluster/apps/application_1404204891930_0002
+      Content-Type: application/xml
+      Server: Jetty(6.1.26)
+
+Response Body:
+
+      No response body
+
+Cluster Application State API
+-----------------------------
+
+With the application state API, you can query the state of a submitted app as well kill a running app by modifying the state of a running app using a PUT request with the state set to "KILLED". To perform the PUT operation, authentication has to be setup for the RM web services. In addition, you must be authorized to kill the app. Currently you can only change the state to "KILLED"; an attempt to change the state to any other results in a 400 error response. Examples of the unauthorized and bad request errors are below. When you carry out a successful PUT, the iniital response may be a 202. You can confirm that the app is killed by repeating the PUT request until you get a 200, querying the state using the GET method or querying for app information and checking the state. In the examples below, we repeat the PUT request and get a 200 response.
+
+Please note that in order to kill an app, you must have an authentication filter setup for the HTTP interface. The functionality requires that a username is set in the HttpServletRequest. If no filter is setup, the response will be an "UNAUTHORIZED" response.
+
+This feature is currently in the alpha stage and may change in the future.
+
+### URI
+
+      * http://<rm http address:port>/ws/v1/cluster/apps/{appid}/state
+
+### HTTP Operations Supported
+
+      * GET
+      * PUT
+
+### Query Parameters Supported
+
+      None
+
+### Elements of *appstate* object
+
+When you make a request for the state of an app, the information returned has the following fields
+
+| Item | Data Type | Description |
+|:---- |:---- |:---- |
+| state | string | The application state - can be one of "NEW", "NEW\_SAVING", "SUBMITTED", "ACCEPTED", "RUNNING", "FINISHED", "FAILED", "KILLED" |
+
+### Response Examples
+
+**JSON responses**
+
+HTTP Request
+
+      GET http://<rm http address:port>/ws/v1/cluster/apps/application_1399397633663_0003/state
+
+Response Header:
+
+    HTTP/1.1 200 OK
+    Content-Type: application/json
+    Transfer-Encoding: chunked
+    Server: Jetty(6.1.26)
+
+Response Body:
+
+    {
+      "state":"ACCEPTED"
+    }
+
+HTTP Request
+
+      PUT http://<rm http address:port>/ws/v1/cluster/apps/application_1399397633663_0003/state
+
+Request Body:
+
+    {
+      "state":"KILLED"
+    }
+
+Response Header:
+
+    HTTP/1.1 202 Accepted
+    Content-Type: application/json
+    Transfer-Encoding: chunked
+    Location: http://<rm http address:port>/ws/v1/cluster/apps/application_1399397633663_0003
+    Server: Jetty(6.1.26)
+
+Response Body:
+
+    {
+      "state":"ACCEPTED"
+    }
+
+      PUT http://<rm http address:port>/ws/v1/cluster/apps/application_1399397633663_0003/state
+
+Request Body:
+
+    {
+      "state":"KILLED"
+    }
+
+Response Header:
+
+    HTTP/1.1 200 OK
+    Content-Type: application/json
+    Transfer-Encoding: chunked
+    Server: Jetty(6.1.26)
+
+Response Body:
+
+    {
+      "state":"KILLED"
+    }
+
+**XML responses**
+
+HTTP Request
+
+      GET http://<rm http address:port>/ws/v1/cluster/apps/application_1399397633663_0003/state
+
+Response Header:
+
+    HTTP/1.1 200 OK
+    Content-Type: application/xml
+    Content-Length: 99
+    Server: Jetty(6.1.26)
+
+Response Body:
+
+    <?xml version="1.0" encoding="UTF-8" standalone="yes"?>
+    <appstate>
+      <state>ACCEPTED</state>
+    </appstate>
+
+HTTP Request
+
+      PUT http://<rm http address:port>/ws/v1/cluster/apps/application_1399397633663_0003/state
+
+Request Body:
+
+    <?xml version="1.0" encoding="UTF-8" standalone="yes"?>
+    <appstate>
+      <state>KILLED</state>
+    </appstate>
+
+Response Header:
+
+    HTTP/1.1 202 Accepted
+    Content-Type: application/xml
+    Content-Length: 794
+    Location: http://<rm http address:port>/ws/v1/cluster/apps/application_1399397633663_0003
+    Server: Jetty(6.1.26)
+
+Response Body:
+
+    <?xml version="1.0" encoding="UTF-8" standalone="yes"?>
+    <appstate>
+      <state>ACCEPTED</state>
+    </appstate>
+
+HTTP Request
+
+      PUT http://<rm http address:port>/ws/v1/cluster/apps/application_1399397633663_0003/state
+
+Request Body:
+
+    <?xml version="1.0" encoding="UTF-8" standalone="yes"?>
+    <appstate>
+      <state>KILLED</state>
+    </appstate>
+
+Response Header:
+
+    HTTP/1.1 200 OK
+    Content-Type: application/xml
+    Content-Length: 917
+    Server: Jetty(6.1.26)
+
+Response Body:
+
+    <?xml version="1.0" encoding="UTF-8" standalone="yes"?>
+    <appstate>
+      <state>KILLED</state>
+    </appstate>
+
+**Unauthorized Error Response**
+
+HTTP Request
+
+      PUT http://<rm http address:port>/ws/v1/cluster/apps/application_1399397633663_0003/state
+
+Request Body:
+
+    <?xml version="1.0" encoding="UTF-8" standalone="yes"?>
+    <appstate>
+      <state>KILLED</state>
+    </appstate>
+
+Response Header:
+
+    HTTP/1.1 403 Unauthorized
+    Server: Jetty(6.1.26)
+
+**Bad Request Error Response**
+
+HTTP Request
+
+      PUT http://<rm http address:port>/ws/v1/cluster/apps/application_1399397633663_0003/state
+
+Request Body:
+
+    <?xml version="1.0" encoding="UTF-8" standalone="yes"?>
+    <appstate>
+      <state>RUNNING</state>
+    </appstate>
+
+Response Header:
+
+    HTTP/1.1 400
+    Content-Length: 295
+    Content-Type: application/xml
+    Server: Jetty(6.1.26)
+
+Response Body:
+
+    <?xml version="1.0" encoding="UTF-8" standalone="yes"?>
+    <RemoteException>
+      <exception>BadRequestException</exception>
+      <message>java.lang.Exception: Only 'KILLED' is allowed as a target state.</message>
+      <javaClassName>org.apache.hadoop.yarn.webapp.BadRequestException</javaClassName>
+    </RemoteException>
+
+Cluster Application Queue API
+-----------------------------
+
+With the application queue API, you can query the queue of a submitted app as well move a running app to another queue using a PUT request specifying the target queue. To perform the PUT operation, authentication has to be setup for the RM web services. In addition, you must be authorized to move the app. Currently you can only move the app if you're using the Capacity scheduler or the Fair scheduler.
+
+Please note that in order to move an app, you must have an authentication filter setup for the HTTP interface. The functionality requires that a username is set in the HttpServletRequest. If no filter is setup, the response will be an "UNAUTHORIZED" response.
+
+This feature is currently in the alpha stage and may change in the future.
+
+### URI
+
+      * http://<rm http address:port>/ws/v1/cluster/apps/{appid}/queue
+
+### HTTP Operations Supported
+
+      * GET
+      * PUT
+
+### Query Parameters Supported
+
+      None
+
+### Elements of *appqueue* object
+
+When you make a request for the state of an app, the information returned has the following fields
+
+| Item | Data Type | Description |
+|:---- |:---- |:---- |
+| queue | string | The application queue |
+
+### Response Examples
+
+**JSON responses**
+
+HTTP Request
+
+      GET http://<rm http address:port>/ws/v1/cluster/apps/application_1399397633663_0003/queue
+
+Response Header:
+
+    HTTP/1.1 200 OK
+    Content-Type: application/json
+    Transfer-Encoding: chunked
+    Server: Jetty(6.1.26)
+
+Response Body:
+
+    {
+      "queue":"default"
+    }
+
+HTTP Request
+
+      PUT http://<rm http address:port>/ws/v1/cluster/apps/application_1399397633663_0003/queue
+
+Request Body:
+
+    {
+      "queue":"test"
+    }
+
+Response Header:
+
+    HTTP/1.1 200 OK
+    Content-Type: application/json
+    Transfer-Encoding: chunked
+    Server: Jetty(6.1.26)
+
+Response Body:
+
+    {
+      "queue":"test"
+    }
+
+**XML responses**
+
+HTTP Request
+
+      GET http://<rm http address:port>/ws/v1/cluster/apps/application_1399397633663_0003/queue
+
+Response Header:
+
+    HTTP/1.1 200 OK
+    Content-Type: application/xml
+    Content-Length: 98
+    Server: Jetty(6.1.26)
+
+Response Body:
+
+    <?xml version="1.0" encoding="UTF-8" standalone="yes"?>
+    <appqueue>
+      <queue>default</queue>
+    </appqueue>
+
+HTTP Request
+
+      PUT http://<rm http address:port>/ws/v1/cluster/apps/application_1399397633663_0003/queue
+
+Request Body:
+
+    <?xml version="1.0" encoding="UTF-8" standalone="yes"?>
+    <appqueue>
+      <queue>test</queue>
+ 

<TRUNCATED>

[04/50] [abbrv] hadoop git commit: YARN-3247. TestQueueMappings should use CapacityScheduler explicitly. Contributed by Zhihai Xu.

Posted by zh...@apache.org.
YARN-3247. TestQueueMappings should use CapacityScheduler explicitly. Contributed by Zhihai Xu.


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

Branch: refs/heads/HDFS-7285
Commit: cadfb71d7492f056a9318069865ab4fcd97d92f8
Parents: ef3702e
Author: Tsuyoshi Ozawa <oz...@apache.org>
Authored: Wed Feb 25 10:34:28 2015 +0900
Committer: Zhe Zhang <zh...@cloudera.com>
Committed: Mon Mar 2 09:13:50 2015 -0800

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                                   | 3 +++
 .../resourcemanager/scheduler/capacity/TestQueueMappings.java     | 2 ++
 2 files changed, 5 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/cadfb71d/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index 39e2dc0..2d11ed7 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -652,6 +652,9 @@ Release 2.7.0 - UNRELEASED
     YARN-3238. Connection timeouts to nodemanagers are retried at
     multiple levels (Jason Lowe via xgong)
 
+    YARN-3247. TestQueueMappings should use CapacityScheduler explicitly.
+    (Zhihai Xu via ozawa)
+
 Release 2.6.0 - 2014-11-18
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cadfb71d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestQueueMappings.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/TestQueueMappings.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/TestQueueMappings.java
index 2317fab..005f40b 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/TestQueueMappings.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/TestQueueMappings.java
@@ -135,6 +135,8 @@ public class TestQueueMappings {
     checkQMapping("a", Q1, cs);
 
     csConf = new CapacitySchedulerConfiguration();
+    csConf.set(YarnConfiguration.RM_SCHEDULER,
+        CapacityScheduler.class.getName());
     setupQueueConfiguration(csConf);
     conf = new YarnConfiguration(csConf);
 


[46/50] [abbrv] hadoop git commit: HADOOP-10774. Update KerberosTestUtils for hadoop-auth tests when using IBM Java (sangamesh via aw)

Posted by zh...@apache.org.
HADOOP-10774. Update KerberosTestUtils for hadoop-auth tests when using IBM Java (sangamesh via aw)


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

Branch: refs/heads/HDFS-7285
Commit: 260883bf402cf8ad9b999a6077ba56e75820738f
Parents: d6c1b62
Author: Allen Wittenauer <aw...@apache.org>
Authored: Sat Feb 28 23:22:06 2015 -0800
Committer: Zhe Zhang <zh...@cloudera.com>
Committed: Mon Mar 2 09:13:55 2015 -0800

----------------------------------------------------------------------
 .../authentication/KerberosTestUtils.java       | 40 ++++++++++++++------
 hadoop-common-project/hadoop-common/CHANGES.txt |  3 ++
 2 files changed, 32 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/260883bf/hadoop-common-project/hadoop-auth/src/test/java/org/apache/hadoop/security/authentication/KerberosTestUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-auth/src/test/java/org/apache/hadoop/security/authentication/KerberosTestUtils.java b/hadoop-common-project/hadoop-auth/src/test/java/org/apache/hadoop/security/authentication/KerberosTestUtils.java
index 7629a30..8fc08e2 100644
--- a/hadoop-common-project/hadoop-auth/src/test/java/org/apache/hadoop/security/authentication/KerberosTestUtils.java
+++ b/hadoop-common-project/hadoop-auth/src/test/java/org/apache/hadoop/security/authentication/KerberosTestUtils.java
@@ -32,12 +32,14 @@ import java.util.Map;
 import java.util.Set;
 import java.util.concurrent.Callable;
 
+import static org.apache.hadoop.util.PlatformName.IBM_JAVA;
+
 /**
  * Test helper class for Java Kerberos setup.
  */
 public class KerberosTestUtils {
   private static String keytabFile = new File(System.getProperty("test.dir", "target"),
-          UUID.randomUUID().toString()).toString();
+          UUID.randomUUID().toString()).getAbsolutePath();
 
   public static String getRealm() {
     return "EXAMPLE.COM";
@@ -65,18 +67,34 @@ public class KerberosTestUtils {
     @Override
     public AppConfigurationEntry[] getAppConfigurationEntry(String name) {
       Map<String, String> options = new HashMap<String, String>();
-      options.put("keyTab", KerberosTestUtils.getKeytabFile());
-      options.put("principal", principal);
-      options.put("useKeyTab", "true");
-      options.put("storeKey", "true");
-      options.put("doNotPrompt", "true");
-      options.put("useTicketCache", "true");
-      options.put("renewTGT", "true");
-      options.put("refreshKrb5Config", "true");
-      options.put("isInitiator", "true");
+      if (IBM_JAVA) {
+        options.put("useKeytab", KerberosTestUtils.getKeytabFile().startsWith("file://") ?   
+                    KerberosTestUtils.getKeytabFile() : "file://" +  KerberosTestUtils.getKeytabFile());
+        options.put("principal", principal);
+        options.put("refreshKrb5Config", "true");
+        options.put("credsType", "both");
+      } else {
+        options.put("keyTab", KerberosTestUtils.getKeytabFile());
+        options.put("principal", principal);
+        options.put("useKeyTab", "true");
+        options.put("storeKey", "true");
+        options.put("doNotPrompt", "true");
+        options.put("useTicketCache", "true");
+        options.put("renewTGT", "true");
+        options.put("refreshKrb5Config", "true");
+        options.put("isInitiator", "true");
+      } 
       String ticketCache = System.getenv("KRB5CCNAME");
       if (ticketCache != null) {
-        options.put("ticketCache", ticketCache);
+        if (IBM_JAVA) {
+          // IBM JAVA only respect system property and not env variable
+          // The first value searched when "useDefaultCcache" is used.
+          System.setProperty("KRB5CCNAME", ticketCache);
+          options.put("useDefaultCcache", "true");
+          options.put("renewTGT", "true");
+        } else {
+          options.put("ticketCache", ticketCache);
+        }
       }
       options.put("debug", "true");
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/260883bf/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 74bf558..3c4dc99 100644
--- a/hadoop-common-project/hadoop-common/CHANGES.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES.txt
@@ -403,6 +403,9 @@ Trunk (Unreleased)
 
     HADOOP-11637. bash location hard-coded in shell scripts (aw)
 
+    HADOOP-10774. Update KerberosTestUtils for hadoop-auth tests when using
+    IBM Java (sangamesh via aw)
+
   OPTIMIZATIONS
 
     HADOOP-7761. Improve the performance of raw comparisons. (todd)


[20/50] [abbrv] hadoop git commit: HDFS-7774. Unresolved symbols error while compiling HDFS on Windows 7/32 bit. Contributed by Kiran Kumar M R.

Posted by zh...@apache.org.
HDFS-7774. Unresolved symbols error while compiling HDFS on Windows 7/32 bit. Contributed by Kiran Kumar M R.


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

Branch: refs/heads/HDFS-7285
Commit: 1a90367c3a6cd708b5f19e71859be864ba95e6db
Parents: 9b0eda1
Author: cnauroth <cn...@apache.org>
Authored: Thu Feb 26 13:37:46 2015 -0800
Committer: Zhe Zhang <zh...@cloudera.com>
Committed: Mon Mar 2 09:13:52 2015 -0800

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt                  | 3 +++
 hadoop-hdfs-project/hadoop-hdfs/pom.xml                      | 5 ++++-
 .../hadoop-hdfs/src/main/native/libhdfs/os/windows/thread.c  | 2 +-
 .../main/native/libhdfs/os/windows/thread_local_storage.c    | 8 ++++++++
 4 files changed, 16 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/1a90367c/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 54b4057..ae83898 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -1040,6 +1040,9 @@ Release 2.7.0 - UNRELEASED
     HDFS-7843. A truncated file is corrupted after rollback from a rolling
     upgrade.  (szetszwo)
 
+    HDFS-7774. Unresolved symbols error while compiling HDFS on Windows 7/32 bit.
+    (Kiran Kumar M R via cnauroth)
+
     BREAKDOWN OF HDFS-7584 SUBTASKS AND RELATED JIRAS
 
       HDFS-7720. Quota by Storage Type API, tools and ClientNameNode

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1a90367c/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 5efce5c..2d402a9 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/pom.xml
+++ b/hadoop-hdfs-project/hadoop-hdfs/pom.xml
@@ -440,10 +440,13 @@ http://maven.apache.org/xsd/maven-4.0.0.xsd">
                 </goals>
                 <configuration>
                   <target>
+                    <condition property="generator" value="Visual Studio 10" else="Visual Studio 10 Win64">
+                      <equals arg1="Win32" arg2="${env.PLATFORM}" />
+                    </condition>
                     <mkdir dir="${project.build.directory}/native"/>
                     <exec executable="cmake" dir="${project.build.directory}/native"
                         failonerror="true">
-                      <arg line="${basedir}/src/ -DGENERATED_JAVAH=${project.build.directory}/native/javah -DJVM_ARCH_DATA_MODEL=${sun.arch.data.model} -DREQUIRE_LIBWEBHDFS=${require.libwebhdfs} -DREQUIRE_FUSE=${require.fuse} -G 'Visual Studio 10 Win64'"/>
+                      <arg line="${basedir}/src/ -DGENERATED_JAVAH=${project.build.directory}/native/javah -DJVM_ARCH_DATA_MODEL=${sun.arch.data.model} -DREQUIRE_LIBWEBHDFS=${require.libwebhdfs} -DREQUIRE_FUSE=${require.fuse} -G '${generator}'"/>
                     </exec>
                     <exec executable="msbuild" dir="${project.build.directory}/native"
                         failonerror="true">

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1a90367c/hadoop-hdfs-project/hadoop-hdfs/src/main/native/libhdfs/os/windows/thread.c
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/native/libhdfs/os/windows/thread.c b/hadoop-hdfs-project/hadoop-hdfs/src/main/native/libhdfs/os/windows/thread.c
index 90450d8..f5cc2a7 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/native/libhdfs/os/windows/thread.c
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/native/libhdfs/os/windows/thread.c
@@ -28,7 +28,7 @@
  * @param toRun thread to run
  * @return DWORD result of running thread (always 0)
  */
-static DWORD runThread(LPVOID toRun) {
+static DWORD WINAPI runThread(LPVOID toRun) {
   const thread *t = toRun;
   t->start(t->arg);
   return 0;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1a90367c/hadoop-hdfs-project/hadoop-hdfs/src/main/native/libhdfs/os/windows/thread_local_storage.c
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/native/libhdfs/os/windows/thread_local_storage.c b/hadoop-hdfs-project/hadoop-hdfs/src/main/native/libhdfs/os/windows/thread_local_storage.c
index 70ad152..4c415e1 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/native/libhdfs/os/windows/thread_local_storage.c
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/native/libhdfs/os/windows/thread_local_storage.c
@@ -96,13 +96,21 @@ static void NTAPI tlsCallback(PVOID h, DWORD reason, PVOID pv)
  * reference _tls_used, we guarantee that the binary retains the TLS directory.
  * See Microsoft Visual Studio 10.0/VC/crt/src/tlssup.c .
  */
+#ifdef _WIN64
 #pragma comment(linker, "/INCLUDE:_tls_used")
+#else
+#pragma comment(linker, "/INCLUDE:__tls_used")
+#endif
 
 /*
  * We must retain a pointer to the callback function.  Force the linker to keep
  * this symbol, even though it appears that nothing in our source code uses it.
  */
+#ifdef _WIN64
 #pragma comment(linker, "/INCLUDE:pTlsCallback")
+#else
+#pragma comment(linker, "/INCLUDE:_pTlsCallback")
+#endif
 
 /*
  * Define constant pointer to our callback, and tell the linker to pin it into


[47/50] [abbrv] hadoop git commit: HDFS-4681. TestBlocksWithNotEnoughRacks#testCorruptBlockRereplicatedAcrossRacks fails using IBM java (Ayappan via aw)

Posted by zh...@apache.org.
HDFS-4681. TestBlocksWithNotEnoughRacks#testCorruptBlockRereplicatedAcrossRacks fails using IBM java (Ayappan via aw)


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

Branch: refs/heads/HDFS-7285
Commit: 133d04cb8972fa3e1e7a7babdc41c21b7340e34c
Parents: 260883b
Author: Allen Wittenauer <aw...@apache.org>
Authored: Sat Feb 28 23:32:09 2015 -0800
Committer: Zhe Zhang <zh...@cloudera.com>
Committed: Mon Mar 2 09:13:55 2015 -0800

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt             |  3 +++
 .../test/java/org/apache/hadoop/hdfs/DFSTestUtil.java   | 12 ++++++++++++
 .../java/org/apache/hadoop/hdfs/MiniDFSCluster.java     | 10 ++++++++++
 .../blockmanagement/TestBlocksWithNotEnoughRacks.java   |  7 ++++---
 4 files changed, 29 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/133d04cb/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 2a8da43..16fe394 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -306,6 +306,9 @@ Trunk (Unreleased)
     HDFS-7803. Wrong command mentioned in HDFSHighAvailabilityWithQJM
     documentation (Arshad Mohammad via aw)
 
+    HDFS-4681. TestBlocksWithNotEnoughRacks#testCorruptBlockRereplicatedAcrossRacks 
+    fails using IBM java (Ayappan via aw)
+
 Release 2.7.0 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/133d04cb/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java
index 5f05d94..c3dac35 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java
@@ -252,6 +252,12 @@ public class DFSTestUtil {
   public void createFiles(FileSystem fs, String topdir) throws IOException {
     createFiles(fs, topdir, (short)3);
   }
+
+  public static byte[] readFileAsBytes(FileSystem fs, Path fileName) throws IOException {
+    ByteArrayOutputStream os = new ByteArrayOutputStream();
+    IOUtils.copyBytes(fs.open(fileName), os, 1024, true);
+    return os.toByteArray();
+  }
   
   /** create nFiles with random names and directory hierarchies
    *  with random (but reproducible) data in them.
@@ -724,6 +730,12 @@ public class DFSTestUtil {
     return b.toString();
   }
 
+  public static byte[] readFileAsBytes(File f) throws IOException {
+    ByteArrayOutputStream os = new ByteArrayOutputStream();
+    IOUtils.copyBytes(new FileInputStream(f), os, 1024, true);
+    return os.toByteArray();
+  }
+
   /* Write the given string to the given file */
   public static void writeFile(FileSystem fs, Path p, String s) 
       throws IOException {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/133d04cb/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSCluster.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSCluster.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSCluster.java
index 5297ba2..2c1d07e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSCluster.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSCluster.java
@@ -1869,6 +1869,16 @@ public class MiniDFSCluster {
     return null;
   }
 
+  public byte[] readBlockOnDataNodeAsBytes(int i, ExtendedBlock block)
+      throws IOException {
+    assert (i >= 0 && i < dataNodes.size()) : "Invalid datanode "+i;
+    File blockFile = getBlockFile(i, block);
+    if (blockFile != null && blockFile.exists()) {
+      return DFSTestUtil.readFileAsBytes(blockFile);
+    }
+    return null;
+  }
+
   /**
    * Corrupt a block on a particular datanode.
    *

http://git-wip-us.apache.org/repos/asf/hadoop/blob/133d04cb/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlocksWithNotEnoughRacks.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlocksWithNotEnoughRacks.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlocksWithNotEnoughRacks.java
index 1bc7cdc..54983a1 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlocksWithNotEnoughRacks.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlocksWithNotEnoughRacks.java
@@ -20,6 +20,7 @@ package org.apache.hadoop.hdfs.server.blockmanagement;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.assertArrayEquals;
 
 import java.util.ArrayList;
 
@@ -202,7 +203,7 @@ public class TestBlocksWithNotEnoughRacks {
       final FileSystem fs = cluster.getFileSystem();
       
       DFSTestUtil.createFile(fs, filePath, fileLen, REPLICATION_FACTOR, 1L);
-      final String fileContent = DFSTestUtil.readFile(fs, filePath);
+      final byte[] fileContent = DFSTestUtil.readFileAsBytes(fs, filePath);
 
       ExtendedBlock b = DFSTestUtil.getFirstBlock(fs, filePath);
       DFSTestUtil.waitForReplication(cluster, b, 2, REPLICATION_FACTOR, 0);
@@ -224,9 +225,9 @@ public class TestBlocksWithNotEnoughRacks {
       // Ensure all replicas are valid (the corrupt replica may not
       // have been cleaned up yet).
       for (int i = 0; i < racks.length; i++) {
-        String blockContent = cluster.readBlockOnDataNode(i, b);
+        byte[] blockContent = cluster.readBlockOnDataNodeAsBytes(i, b);
         if (blockContent != null && i != dnToCorrupt) {
-          assertEquals("Corrupt replica", fileContent, blockContent);
+          assertArrayEquals("Corrupt replica", fileContent, blockContent);
         }
       }
     } finally {


[10/50] [abbrv] hadoop git commit: HADOOP-11620. Add support for load balancing across a group of KMS for HA. Contributed by Arun Suresh.

Posted by zh...@apache.org.
HADOOP-11620. Add support for load balancing across a group of KMS for HA. 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/9f356cfa
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/9f356cfa
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/9f356cfa

Branch: refs/heads/HDFS-7285
Commit: 9f356cfaf589ff55842c180be0adee948d8aa060
Parents: 0127820
Author: Andrew Wang <wa...@apache.org>
Authored: Wed Feb 25 21:15:44 2015 -0800
Committer: Zhe Zhang <zh...@cloudera.com>
Committed: Mon Mar 2 09:13:51 2015 -0800

----------------------------------------------------------------------
 hadoop-common-project/hadoop-common/CHANGES.txt |   3 +
 .../crypto/key/kms/KMSClientProvider.java       |  84 ++++-
 .../key/kms/LoadBalancingKMSClientProvider.java | 347 +++++++++++++++++++
 .../kms/TestLoadBalancingKMSClientProvider.java | 166 +++++++++
 .../hadoop/crypto/key/kms/server/TestKMS.java   | 114 +++---
 5 files changed, 654 insertions(+), 60 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/9f356cfa/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 0d452f7..39062a8 100644
--- a/hadoop-common-project/hadoop-common/CHANGES.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES.txt
@@ -648,6 +648,9 @@ Release 2.7.0 - UNRELEASED
     HADOOP-11506. Configuration variable expansion regex expensive for long
     values. (Gera Shegalov via gera)
 
+    HADOOP-11620. Add support for load balancing across a group of KMS for HA.
+    (Arun Suresh via wang)
+
   BUG FIXES
 
     HADOOP-11512. Use getTrimmedStrings when reading serialization keys

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9f356cfa/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/kms/KMSClientProvider.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/kms/KMSClientProvider.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/kms/KMSClientProvider.java
index 97ab253..223e69a 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/kms/KMSClientProvider.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/kms/KMSClientProvider.java
@@ -52,6 +52,7 @@ import java.io.Writer;
 import java.lang.reflect.UndeclaredThrowableException;
 import java.net.HttpURLConnection;
 import java.net.InetSocketAddress;
+import java.net.MalformedURLException;
 import java.net.SocketTimeoutException;
 import java.net.URI;
 import java.net.URISyntaxException;
@@ -74,6 +75,7 @@ import org.apache.hadoop.crypto.key.KeyProviderCryptoExtension.CryptoExtension;
 
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
+import com.google.common.base.Strings;
 
 /**
  * KMS client <code>KeyProvider</code> implementation.
@@ -221,14 +223,71 @@ public class KMSClientProvider extends KeyProvider implements CryptoExtension,
    */
   public static class Factory extends KeyProviderFactory {
 
+    /**
+     * This provider expects URIs in the following form :
+     * kms://<PROTO>@<AUTHORITY>/<PATH>
+     *
+     * where :
+     * - PROTO = http or https
+     * - AUTHORITY = <HOSTS>[:<PORT>]
+     * - HOSTS = <HOSTNAME>[;<HOSTS>]
+     * - HOSTNAME = string
+     * - PORT = integer
+     *
+     * If multiple hosts are provider, the Factory will create a
+     * {@link LoadBalancingKMSClientProvider} that round-robins requests
+     * across the provided list of hosts.
+     */
     @Override
-    public KeyProvider createProvider(URI providerName, Configuration conf)
+    public KeyProvider createProvider(URI providerUri, Configuration conf)
         throws IOException {
-      if (SCHEME_NAME.equals(providerName.getScheme())) {
-        return new KMSClientProvider(providerName, conf);
+      if (SCHEME_NAME.equals(providerUri.getScheme())) {
+        URL origUrl = new URL(extractKMSPath(providerUri).toString());
+        String authority = origUrl.getAuthority();
+        // check for ';' which delimits the backup hosts
+        if (Strings.isNullOrEmpty(authority)) {
+          throw new IOException(
+              "No valid authority in kms uri [" + origUrl + "]");
+        }
+        // Check if port is present in authority
+        // In the current scheme, all hosts have to run on the same port
+        int port = -1;
+        String hostsPart = authority;
+        if (authority.contains(":")) {
+          String[] t = authority.split(":");
+          try {
+            port = Integer.parseInt(t[1]);
+          } catch (Exception e) {
+            throw new IOException(
+                "Could not parse port in kms uri [" + origUrl + "]");
+          }
+          hostsPart = t[0];
+        }
+        return createProvider(providerUri, conf, origUrl, port, hostsPart);
       }
       return null;
     }
+
+    private KeyProvider createProvider(URI providerUri, Configuration conf,
+        URL origUrl, int port, String hostsPart) throws IOException {
+      String[] hosts = hostsPart.split(";");
+      if (hosts.length == 1) {
+        return new KMSClientProvider(providerUri, conf);
+      } else {
+        KMSClientProvider[] providers = new KMSClientProvider[hosts.length];
+        for (int i = 0; i < hosts.length; i++) {
+          try {
+            providers[i] =
+                new KMSClientProvider(
+                    new URI("kms", origUrl.getProtocol(), hosts[i], port,
+                        origUrl.getPath(), null, null), conf);
+          } catch (URISyntaxException e) {
+            throw new IOException("Could not instantiate KMSProvider..", e);
+          }
+        }
+        return new LoadBalancingKMSClientProvider(providers, conf);
+      }
+    }
   }
 
   public static <T> T checkNotNull(T o, String name)
@@ -302,10 +361,8 @@ public class KMSClientProvider extends KeyProvider implements CryptoExtension,
 
   public KMSClientProvider(URI uri, Configuration conf) throws IOException {
     super(conf);
-    Path path = ProviderUtils.unnestUri(uri);
-    URL url = path.toUri().toURL();
-    kmsUrl = createServiceURL(url);
-    if ("https".equalsIgnoreCase(url.getProtocol())) {
+    kmsUrl = createServiceURL(extractKMSPath(uri));
+    if ("https".equalsIgnoreCase(new URL(kmsUrl).getProtocol())) {
       sslFactory = new SSLFactory(SSLFactory.Mode.CLIENT, conf);
       try {
         sslFactory.init();
@@ -346,8 +403,12 @@ public class KMSClientProvider extends KeyProvider implements CryptoExtension,
             .getCurrentUser();
   }
 
-  private String createServiceURL(URL url) throws IOException {
-    String str = url.toExternalForm();
+  private static Path extractKMSPath(URI uri) throws MalformedURLException, IOException {
+    return ProviderUtils.unnestUri(uri);
+  }
+
+  private static String createServiceURL(Path path) throws IOException {
+    String str = new URL(path.toString()).toExternalForm();
     if (str.endsWith("/")) {
       str = str.substring(0, str.length() - 1);
     }
@@ -853,4 +914,9 @@ public class KMSClientProvider extends KeyProvider implements CryptoExtension,
       }
     }
   }
+
+  @VisibleForTesting
+  String getKMSUrl() {
+    return kmsUrl;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9f356cfa/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/kms/LoadBalancingKMSClientProvider.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/kms/LoadBalancingKMSClientProvider.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/kms/LoadBalancingKMSClientProvider.java
new file mode 100644
index 0000000..c1579e7
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/kms/LoadBalancingKMSClientProvider.java
@@ -0,0 +1,347 @@
+/**
+ * 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.crypto.key.kms;
+
+import java.io.IOException;
+import java.security.GeneralSecurityException;
+import java.security.NoSuchAlgorithmException;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.crypto.key.KeyProvider;
+import org.apache.hadoop.crypto.key.KeyProviderCryptoExtension.CryptoExtension;
+import org.apache.hadoop.crypto.key.KeyProviderCryptoExtension.EncryptedKeyVersion;
+import org.apache.hadoop.crypto.key.KeyProviderDelegationTokenExtension;
+import org.apache.hadoop.security.Credentials;
+import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.util.Time;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.annotations.VisibleForTesting;
+
+/**
+ * A simple LoadBalancing KMSClientProvider that round-robins requests
+ * across a provided array of KMSClientProviders. It also retries failed
+ * requests on the next available provider in the load balancer group. It
+ * only retries failed requests that result in an IOException, sending back
+ * all other Exceptions to the caller without retry.
+ */
+public class LoadBalancingKMSClientProvider extends KeyProvider implements
+    CryptoExtension,
+    KeyProviderDelegationTokenExtension.DelegationTokenExtension {
+
+  public static Logger LOG =
+      LoggerFactory.getLogger(LoadBalancingKMSClientProvider.class);
+
+  static interface ProviderCallable<T> {
+    public T call(KMSClientProvider provider) throws IOException, Exception;
+  }
+
+  @SuppressWarnings("serial")
+  static class WrapperException extends RuntimeException {
+    public WrapperException(Throwable cause) {
+      super(cause);
+    }
+  }
+
+  private final KMSClientProvider[] providers;
+  private final AtomicInteger currentIdx;
+
+  public LoadBalancingKMSClientProvider(KMSClientProvider[] providers,
+      Configuration conf) {
+    this(shuffle(providers), Time.monotonicNow(), conf);
+  }
+
+  @VisibleForTesting
+  LoadBalancingKMSClientProvider(KMSClientProvider[] providers, long seed,
+      Configuration conf) {
+    super(conf);
+    this.providers = providers;
+    this.currentIdx = new AtomicInteger((int)(seed % providers.length));
+  }
+
+  @VisibleForTesting
+  KMSClientProvider[] getProviders() {
+    return providers;
+  }
+
+  private <T> T doOp(ProviderCallable<T> op, int currPos)
+      throws IOException {
+    IOException ex = null;
+    for (int i = 0; i < providers.length; i++) {
+      KMSClientProvider provider = providers[(currPos + i) % providers.length];
+      try {
+        return op.call(provider);
+      } catch (IOException ioe) {
+        LOG.warn("KMS provider at [{}] threw an IOException [{}]!!",
+            provider.getKMSUrl(), ioe.getMessage());
+        ex = ioe;
+      } catch (Exception e) {
+        if (e instanceof RuntimeException) {
+          throw (RuntimeException)e;
+        } else {
+          throw new WrapperException(e);
+        }
+      }
+    }
+    if (ex != null) {
+      LOG.warn("Aborting since the Request has failed with all KMS"
+          + " providers in the group. !!");
+      throw ex;
+    }
+    throw new IOException("No providers configured !!");
+  }
+
+  private int nextIdx() {
+    while (true) {
+      int current = currentIdx.get();
+      int next = (current + 1) % providers.length;
+      if (currentIdx.compareAndSet(current, next)) {
+        return current;
+      }
+    }
+  }
+
+  @Override
+  public Token<?>[]
+      addDelegationTokens(final String renewer, final Credentials credentials)
+          throws IOException {
+    return doOp(new ProviderCallable<Token<?>[]>() {
+      @Override
+      public Token<?>[] call(KMSClientProvider provider) throws IOException {
+        return provider.addDelegationTokens(renewer, credentials);
+      }
+    }, nextIdx());
+  }
+
+  // This request is sent to all providers in the load-balancing group
+  @Override
+  public void warmUpEncryptedKeys(String... keyNames) throws IOException {
+    for (KMSClientProvider provider : providers) {
+      try {
+        provider.warmUpEncryptedKeys(keyNames);
+      } catch (IOException ioe) {
+        LOG.error(
+            "Error warming up keys for provider with url"
+            + "[" + provider.getKMSUrl() + "]");
+      }
+    }
+  }
+
+  // This request is sent to all providers in the load-balancing group
+  @Override
+  public void drain(String keyName) {
+    for (KMSClientProvider provider : providers) {
+      provider.drain(keyName);
+    }
+  }
+
+  @Override
+  public EncryptedKeyVersion
+      generateEncryptedKey(final String encryptionKeyName)
+          throws IOException, GeneralSecurityException {
+    try {
+      return doOp(new ProviderCallable<EncryptedKeyVersion>() {
+        @Override
+        public EncryptedKeyVersion call(KMSClientProvider provider)
+            throws IOException, GeneralSecurityException {
+          return provider.generateEncryptedKey(encryptionKeyName);
+        }
+      }, nextIdx());
+    } catch (WrapperException we) {
+      throw (GeneralSecurityException) we.getCause();
+    }
+  }
+
+  @Override
+  public KeyVersion
+      decryptEncryptedKey(final EncryptedKeyVersion encryptedKeyVersion)
+          throws IOException, GeneralSecurityException {
+    try {
+      return doOp(new ProviderCallable<KeyVersion>() {
+        @Override
+        public KeyVersion call(KMSClientProvider provider)
+            throws IOException, GeneralSecurityException {
+          return provider.decryptEncryptedKey(encryptedKeyVersion);
+        }
+      }, nextIdx());
+    } catch (WrapperException we) {
+      throw (GeneralSecurityException)we.getCause();
+    }
+  }
+
+  @Override
+  public KeyVersion getKeyVersion(final String versionName) throws IOException {
+    return doOp(new ProviderCallable<KeyVersion>() {
+      @Override
+      public KeyVersion call(KMSClientProvider provider) throws IOException {
+        return provider.getKeyVersion(versionName);
+      }
+    }, nextIdx());
+  }
+
+  @Override
+  public List<String> getKeys() throws IOException {
+    return doOp(new ProviderCallable<List<String>>() {
+      @Override
+      public List<String> call(KMSClientProvider provider) throws IOException {
+        return provider.getKeys();
+      }
+    }, nextIdx());
+  }
+
+  @Override
+  public Metadata[] getKeysMetadata(final String... names) throws IOException {
+    return doOp(new ProviderCallable<Metadata[]>() {
+      @Override
+      public Metadata[] call(KMSClientProvider provider) throws IOException {
+        return provider.getKeysMetadata(names);
+      }
+    }, nextIdx());
+  }
+
+  @Override
+  public List<KeyVersion> getKeyVersions(final String name) throws IOException {
+    return doOp(new ProviderCallable<List<KeyVersion>>() {
+      @Override
+      public List<KeyVersion> call(KMSClientProvider provider)
+          throws IOException {
+        return provider.getKeyVersions(name);
+      }
+    }, nextIdx());
+  }
+
+  @Override
+  public KeyVersion getCurrentKey(final String name) throws IOException {
+    return doOp(new ProviderCallable<KeyVersion>() {
+      @Override
+      public KeyVersion call(KMSClientProvider provider) throws IOException {
+        return provider.getCurrentKey(name);
+      }
+    }, nextIdx());
+  }
+  @Override
+  public Metadata getMetadata(final String name) throws IOException {
+    return doOp(new ProviderCallable<Metadata>() {
+      @Override
+      public Metadata call(KMSClientProvider provider) throws IOException {
+        return provider.getMetadata(name);
+      }
+    }, nextIdx());
+  }
+
+  @Override
+  public KeyVersion createKey(final String name, final byte[] material,
+      final Options options) throws IOException {
+    return doOp(new ProviderCallable<KeyVersion>() {
+      @Override
+      public KeyVersion call(KMSClientProvider provider) throws IOException {
+        return provider.createKey(name, material, options);
+      }
+    }, nextIdx());
+  }
+
+  @Override
+  public KeyVersion createKey(final String name, final Options options)
+      throws NoSuchAlgorithmException, IOException {
+    try {
+      return doOp(new ProviderCallable<KeyVersion>() {
+        @Override
+        public KeyVersion call(KMSClientProvider provider) throws IOException,
+            NoSuchAlgorithmException {
+          return provider.createKey(name, options);
+        }
+      }, nextIdx());
+    } catch (WrapperException e) {
+      throw (NoSuchAlgorithmException)e.getCause();
+    }
+  }
+  @Override
+  public void deleteKey(final String name) throws IOException {
+    doOp(new ProviderCallable<Void>() {
+      @Override
+      public Void call(KMSClientProvider provider) throws IOException {
+        provider.deleteKey(name);
+        return null;
+      }
+    }, nextIdx());
+  }
+  @Override
+  public KeyVersion rollNewVersion(final String name, final byte[] material)
+      throws IOException {
+    return doOp(new ProviderCallable<KeyVersion>() {
+      @Override
+      public KeyVersion call(KMSClientProvider provider) throws IOException {
+        return provider.rollNewVersion(name, material);
+      }
+    }, nextIdx());
+  }
+
+  @Override
+  public KeyVersion rollNewVersion(final String name)
+      throws NoSuchAlgorithmException, IOException {
+    try {
+      return doOp(new ProviderCallable<KeyVersion>() {
+        @Override
+        public KeyVersion call(KMSClientProvider provider) throws IOException,
+        NoSuchAlgorithmException {
+          return provider.rollNewVersion(name);
+        }
+      }, nextIdx());
+    } catch (WrapperException e) {
+      throw (NoSuchAlgorithmException)e.getCause();
+    }
+  }
+
+  // Close all providers in the LB group
+  @Override
+  public void close() throws IOException {
+    for (KMSClientProvider provider : providers) {
+      try {
+        provider.close();
+      } catch (IOException ioe) {
+        LOG.error("Error closing provider with url"
+            + "[" + provider.getKMSUrl() + "]");
+      }
+    }
+  }
+
+
+  @Override
+  public void flush() throws IOException {
+    for (KMSClientProvider provider : providers) {
+      try {
+        provider.flush();
+      } catch (IOException ioe) {
+        LOG.error("Error flushing provider with url"
+            + "[" + provider.getKMSUrl() + "]");
+      }
+    }
+  }
+
+  private static KMSClientProvider[] shuffle(KMSClientProvider[] providers) {
+    List<KMSClientProvider> list = Arrays.asList(providers);
+    Collections.shuffle(list);
+    return list.toArray(providers);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9f356cfa/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/crypto/key/kms/TestLoadBalancingKMSClientProvider.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/crypto/key/kms/TestLoadBalancingKMSClientProvider.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/crypto/key/kms/TestLoadBalancingKMSClientProvider.java
new file mode 100644
index 0000000..08a3d93
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/crypto/key/kms/TestLoadBalancingKMSClientProvider.java
@@ -0,0 +1,166 @@
+/**    when(p1.getKMSUrl()).thenReturn("p1");
+ * 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.crypto.key.kms;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+import java.io.IOException;
+import java.net.URI;
+import java.security.NoSuchAlgorithmException;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.crypto.key.KeyProvider;
+import org.apache.hadoop.crypto.key.KeyProvider.Options;
+import org.junit.Test;
+import org.mockito.Mockito;
+
+import com.google.common.collect.Sets;
+
+public class TestLoadBalancingKMSClientProvider {
+
+  @Test
+  public void testCreation() throws Exception {
+    Configuration conf = new Configuration();
+    KeyProvider kp = new KMSClientProvider.Factory().createProvider(new URI(
+        "kms://http@host1/kms/foo"), conf);
+    assertTrue(kp instanceof KMSClientProvider);
+    assertEquals("http://host1/kms/foo/v1/",
+        ((KMSClientProvider) kp).getKMSUrl());
+
+    kp = new KMSClientProvider.Factory().createProvider(new URI(
+        "kms://http@host1;host2;host3/kms/foo"), conf);
+    assertTrue(kp instanceof LoadBalancingKMSClientProvider);
+    KMSClientProvider[] providers =
+        ((LoadBalancingKMSClientProvider) kp).getProviders();
+    assertEquals(3, providers.length);
+    assertEquals(Sets.newHashSet("http://host1/kms/foo/v1/",
+        "http://host2/kms/foo/v1/",
+        "http://host3/kms/foo/v1/"),
+        Sets.newHashSet(providers[0].getKMSUrl(),
+            providers[1].getKMSUrl(),
+            providers[2].getKMSUrl()));
+
+    kp = new KMSClientProvider.Factory().createProvider(new URI(
+        "kms://http@host1;host2;host3:16000/kms/foo"), conf);
+    assertTrue(kp instanceof LoadBalancingKMSClientProvider);
+    providers =
+        ((LoadBalancingKMSClientProvider) kp).getProviders();
+    assertEquals(3, providers.length);
+    assertEquals(Sets.newHashSet("http://host1:16000/kms/foo/v1/",
+        "http://host2:16000/kms/foo/v1/",
+        "http://host3:16000/kms/foo/v1/"),
+        Sets.newHashSet(providers[0].getKMSUrl(),
+            providers[1].getKMSUrl(),
+            providers[2].getKMSUrl()));
+  }
+
+  @Test
+  public void testLoadBalancing() throws Exception {
+    Configuration conf = new Configuration();
+    KMSClientProvider p1 = mock(KMSClientProvider.class);
+    when(p1.createKey(Mockito.anyString(), Mockito.any(Options.class)))
+        .thenReturn(
+            new KMSClientProvider.KMSKeyVersion("p1", "v1", new byte[0]));
+    KMSClientProvider p2 = mock(KMSClientProvider.class);
+    when(p2.createKey(Mockito.anyString(), Mockito.any(Options.class)))
+        .thenReturn(
+            new KMSClientProvider.KMSKeyVersion("p2", "v2", new byte[0]));
+    KMSClientProvider p3 = mock(KMSClientProvider.class);
+    when(p3.createKey(Mockito.anyString(), Mockito.any(Options.class)))
+        .thenReturn(
+            new KMSClientProvider.KMSKeyVersion("p3", "v3", new byte[0]));
+    KeyProvider kp = new LoadBalancingKMSClientProvider(
+        new KMSClientProvider[] { p1, p2, p3 }, 0, conf);
+    assertEquals("p1", kp.createKey("test1", new Options(conf)).getName());
+    assertEquals("p2", kp.createKey("test2", new Options(conf)).getName());
+    assertEquals("p3", kp.createKey("test3", new Options(conf)).getName());
+    assertEquals("p1", kp.createKey("test4", new Options(conf)).getName());
+  }
+
+  @Test
+  public void testLoadBalancingWithFailure() throws Exception {
+    Configuration conf = new Configuration();
+    KMSClientProvider p1 = mock(KMSClientProvider.class);
+    when(p1.createKey(Mockito.anyString(), Mockito.any(Options.class)))
+        .thenReturn(
+            new KMSClientProvider.KMSKeyVersion("p1", "v1", new byte[0]));
+    when(p1.getKMSUrl()).thenReturn("p1");
+    // This should not be retried
+    KMSClientProvider p2 = mock(KMSClientProvider.class);
+    when(p2.createKey(Mockito.anyString(), Mockito.any(Options.class)))
+        .thenThrow(new NoSuchAlgorithmException("p2"));
+    when(p2.getKMSUrl()).thenReturn("p2");
+    KMSClientProvider p3 = mock(KMSClientProvider.class);
+    when(p3.createKey(Mockito.anyString(), Mockito.any(Options.class)))
+        .thenReturn(
+            new KMSClientProvider.KMSKeyVersion("p3", "v3", new byte[0]));
+    when(p3.getKMSUrl()).thenReturn("p3");
+    // This should be retried
+    KMSClientProvider p4 = mock(KMSClientProvider.class);
+    when(p4.createKey(Mockito.anyString(), Mockito.any(Options.class)))
+        .thenThrow(new IOException("p4"));
+    when(p4.getKMSUrl()).thenReturn("p4");
+    KeyProvider kp = new LoadBalancingKMSClientProvider(
+        new KMSClientProvider[] { p1, p2, p3, p4 }, 0, conf);
+
+    assertEquals("p1", kp.createKey("test4", new Options(conf)).getName());
+    // Exceptions other than IOExceptions will not be retried
+    try {
+      kp.createKey("test1", new Options(conf)).getName();
+      fail("Should fail since its not an IOException");
+    } catch (Exception e) {
+      assertTrue(e instanceof NoSuchAlgorithmException);
+    }
+    assertEquals("p3", kp.createKey("test2", new Options(conf)).getName());
+    // IOException will trigger retry in next provider
+    assertEquals("p1", kp.createKey("test3", new Options(conf)).getName());
+  }
+
+  @Test
+  public void testLoadBalancingWithAllBadNodes() throws Exception {
+    Configuration conf = new Configuration();
+    KMSClientProvider p1 = mock(KMSClientProvider.class);
+    when(p1.createKey(Mockito.anyString(), Mockito.any(Options.class)))
+        .thenThrow(new IOException("p1"));
+    KMSClientProvider p2 = mock(KMSClientProvider.class);
+    when(p2.createKey(Mockito.anyString(), Mockito.any(Options.class)))
+        .thenThrow(new IOException("p2"));
+    KMSClientProvider p3 = mock(KMSClientProvider.class);
+    when(p3.createKey(Mockito.anyString(), Mockito.any(Options.class)))
+        .thenThrow(new IOException("p3"));
+    KMSClientProvider p4 = mock(KMSClientProvider.class);
+    when(p4.createKey(Mockito.anyString(), Mockito.any(Options.class)))
+        .thenThrow(new IOException("p4"));
+    when(p1.getKMSUrl()).thenReturn("p1");
+    when(p2.getKMSUrl()).thenReturn("p2");
+    when(p3.getKMSUrl()).thenReturn("p3");
+    when(p4.getKMSUrl()).thenReturn("p4");
+    KeyProvider kp = new LoadBalancingKMSClientProvider(
+        new KMSClientProvider[] { p1, p2, p3, p4 }, 0, conf);
+    try {
+      kp.createKey("test3", new Options(conf)).getName();
+      fail("Should fail since all providers threw an IOException");
+    } catch (Exception e) {
+      assertTrue(e instanceof IOException);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9f356cfa/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 70ba95f..c5a990b 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
@@ -24,9 +24,11 @@ import org.apache.hadoop.crypto.key.KeyProvider;
 import org.apache.hadoop.crypto.key.KeyProvider.KeyVersion;
 import org.apache.hadoop.crypto.key.KeyProvider.Options;
 import org.apache.hadoop.crypto.key.KeyProviderCryptoExtension;
+import org.apache.hadoop.crypto.key.KeyProviderCryptoExtension.CryptoExtension;
 import org.apache.hadoop.crypto.key.KeyProviderCryptoExtension.EncryptedKeyVersion;
 import org.apache.hadoop.crypto.key.KeyProviderDelegationTokenExtension;
 import org.apache.hadoop.crypto.key.kms.KMSClientProvider;
+import org.apache.hadoop.crypto.key.kms.LoadBalancingKMSClientProvider;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.minikdc.MiniKdc;
@@ -99,6 +101,12 @@ public class TestKMS {
     }
   }
 
+  protected KeyProvider createProvider(URI uri, Configuration conf)
+      throws IOException {
+    return new LoadBalancingKMSClientProvider(
+        new KMSClientProvider[] { new KMSClientProvider(uri, conf) }, conf);
+  }
+
   protected <T> T runServer(String keystore, String password, File confDir,
       KMSCallable<T> callable) throws Exception {
     return runServer(-1, keystore, password, confDir, callable);
@@ -305,7 +313,7 @@ public class TestKMS {
         final URI uri = createKMSUri(getKMSUrl());
 
         if (ssl) {
-          KeyProvider testKp = new KMSClientProvider(uri, conf);
+          KeyProvider testKp = createProvider(uri, conf);
           ThreadGroup threadGroup = Thread.currentThread().getThreadGroup();
           while (threadGroup.getParent() != null) {
             threadGroup = threadGroup.getParent();
@@ -335,12 +343,14 @@ public class TestKMS {
             doAs(user, new PrivilegedExceptionAction<Void>() {
               @Override
               public Void run() throws Exception {
-                final KeyProvider kp = new KMSClientProvider(uri, conf);
+                final KeyProvider kp = createProvider(uri, conf);
                 // getKeys() empty
                 Assert.assertTrue(kp.getKeys().isEmpty());
 
                 Thread.sleep(4000);
-                Token<?>[] tokens = ((KMSClientProvider)kp).addDelegationTokens("myuser", new Credentials());
+                Token<?>[] tokens =
+                    ((KeyProviderDelegationTokenExtension.DelegationTokenExtension)kp)
+                    .addDelegationTokens("myuser", new Credentials());
                 Assert.assertEquals(1, tokens.length);
                 Assert.assertEquals("kms-dt", tokens[0].getKind().toString());
                 return null;
@@ -348,12 +358,14 @@ public class TestKMS {
             });
           }
         } else {
-          KeyProvider kp = new KMSClientProvider(uri, conf);
+          KeyProvider kp = createProvider(uri, conf);
           // getKeys() empty
           Assert.assertTrue(kp.getKeys().isEmpty());
 
           Thread.sleep(4000);
-          Token<?>[] tokens = ((KMSClientProvider)kp).addDelegationTokens("myuser", new Credentials());
+          Token<?>[] tokens =
+              ((KeyProviderDelegationTokenExtension.DelegationTokenExtension)kp)
+              .addDelegationTokens("myuser", new Credentials());
           Assert.assertEquals(1, tokens.length);
           Assert.assertEquals("kms-dt", tokens[0].getKind().toString());
         }
@@ -404,7 +416,7 @@ public class TestKMS {
         Date started = new Date();
         Configuration conf = new Configuration();
         URI uri = createKMSUri(getKMSUrl());
-        KeyProvider kp = new KMSClientProvider(uri, conf);
+        KeyProvider kp = createProvider(uri, conf);
 
         // getKeys() empty
         Assert.assertTrue(kp.getKeys().isEmpty());
@@ -687,7 +699,7 @@ public class TestKMS {
         doAs("CREATE", new PrivilegedExceptionAction<Void>() {
           @Override
           public Void run() throws Exception {
-            KeyProvider kp = new KMSClientProvider(uri, conf);
+            KeyProvider kp = createProvider(uri, conf);
             try {
               Options options = new KeyProvider.Options(conf);
               Map<String, String> attributes = options.getAttributes();
@@ -727,7 +739,7 @@ public class TestKMS {
         doAs("DECRYPT_EEK", new PrivilegedExceptionAction<Void>() {
           @Override
           public Void run() throws Exception {
-            KeyProvider kp = new KMSClientProvider(uri, conf);
+            KeyProvider kp = createProvider(uri, conf);
             try {
               Options options = new KeyProvider.Options(conf);
               Map<String, String> attributes = options.getAttributes();
@@ -760,7 +772,7 @@ public class TestKMS {
         doAs("ROLLOVER", new PrivilegedExceptionAction<Void>() {
           @Override
           public Void run() throws Exception {
-            KeyProvider kp = new KMSClientProvider(uri, conf);
+            KeyProvider kp = createProvider(uri, conf);
             try {
               Options options = new KeyProvider.Options(conf);
               Map<String, String> attributes = options.getAttributes();
@@ -804,7 +816,7 @@ public class TestKMS {
         doAs("GET", new PrivilegedExceptionAction<Void>() {
           @Override
           public Void run() throws Exception {
-            KeyProvider kp = new KMSClientProvider(uri, conf);
+            KeyProvider kp = createProvider(uri, conf);
             try {
               Options options = new KeyProvider.Options(conf);
               Map<String, String> attributes = options.getAttributes();
@@ -836,7 +848,7 @@ public class TestKMS {
         final EncryptedKeyVersion ekv = doAs("GENERATE_EEK", new PrivilegedExceptionAction<EncryptedKeyVersion>() {
           @Override
           public EncryptedKeyVersion run() throws Exception {
-            KeyProvider kp = new KMSClientProvider(uri, conf);
+            KeyProvider kp = createProvider(uri, conf);
             try {
               Options options = new KeyProvider.Options(conf);
               Map<String, String> attributes = options.getAttributes();
@@ -861,7 +873,7 @@ public class TestKMS {
         doAs("ROLLOVER", new PrivilegedExceptionAction<Void>() {
           @Override
           public Void run() throws Exception {
-            KeyProvider kp = new KMSClientProvider(uri, conf);
+            KeyProvider kp = createProvider(uri, conf);
             try {
               KeyProviderCryptoExtension kpce =
                   KeyProviderCryptoExtension.createKeyProviderCryptoExtension(kp);
@@ -891,7 +903,7 @@ public class TestKMS {
         doAs("GENERATE_EEK", new PrivilegedExceptionAction<Void>() {
           @Override
           public Void run() throws Exception {
-            KeyProvider kp = new KMSClientProvider(uri, conf);
+            KeyProvider kp = createProvider(uri, conf);
             try {
               KeyProviderCryptoExtension kpce =
                   KeyProviderCryptoExtension.createKeyProviderCryptoExtension(kp);
@@ -964,7 +976,7 @@ public class TestKMS {
                 new PrivilegedExceptionAction<KeyProvider>() {
                   @Override
                   public KeyProvider run() throws Exception {
-                    KMSClientProvider kp = new KMSClientProvider(uri, conf);
+                    KeyProvider kp = createProvider(uri, conf);
                         kp.createKey("k1", new byte[16],
                             new KeyProvider.Options(conf));
                     return kp;
@@ -1041,7 +1053,7 @@ public class TestKMS {
                 new PrivilegedExceptionAction<Void>() {
                   @Override
                   public Void run() throws Exception {
-                    KMSClientProvider kp = new KMSClientProvider(uri, conf);
+                    KeyProvider kp = createProvider(uri, conf);
 
                     kp.createKey("k0", new byte[16],
                         new KeyProvider.Options(conf));
@@ -1072,7 +1084,7 @@ public class TestKMS {
                 new PrivilegedExceptionAction<Void>() {
                   @Override
                   public Void run() throws Exception {
-                    KMSClientProvider kp = new KMSClientProvider(uri, conf);
+                    KeyProvider kp = createProvider(uri, conf);
                     kp.createKey("k3", new byte[16],
                         new KeyProvider.Options(conf));
                     // Atleast 2 rollovers.. so should induce signer Exception
@@ -1132,7 +1144,7 @@ public class TestKMS {
         doAs("client", new PrivilegedExceptionAction<Void>() {
           @Override
           public Void run() throws Exception {
-            KeyProvider kp = new KMSClientProvider(uri, conf);
+            KeyProvider kp = createProvider(uri, conf);
             try {
               kp.createKey("k", new KeyProvider.Options(conf));
               Assert.fail();
@@ -1223,7 +1235,7 @@ public class TestKMS {
         doAs("CREATE", new PrivilegedExceptionAction<Void>() {
           @Override
           public Void run() throws Exception {
-            KeyProvider kp = new KMSClientProvider(uri, conf);
+            KeyProvider kp = createProvider(uri, conf);
             try {
               KeyProvider.KeyVersion kv = kp.createKey("k0",
                   new KeyProvider.Options(conf));
@@ -1238,7 +1250,7 @@ public class TestKMS {
         doAs("DELETE", new PrivilegedExceptionAction<Void>() {
           @Override
           public Void run() throws Exception {
-            KeyProvider kp = new KMSClientProvider(uri, conf);
+            KeyProvider kp = createProvider(uri, conf);
             try {
               kp.deleteKey("k0");
             } catch (Exception ex) {
@@ -1251,7 +1263,7 @@ public class TestKMS {
         doAs("SET_KEY_MATERIAL", new PrivilegedExceptionAction<Void>() {
           @Override
           public Void run() throws Exception {
-            KeyProvider kp = new KMSClientProvider(uri, conf);
+            KeyProvider kp = createProvider(uri, conf);
             try {
               KeyProvider.KeyVersion kv = kp.createKey("k1", new byte[16],
                   new KeyProvider.Options(conf));
@@ -1266,7 +1278,7 @@ public class TestKMS {
         doAs("ROLLOVER", new PrivilegedExceptionAction<Void>() {
           @Override
           public Void run() throws Exception {
-            KeyProvider kp = new KMSClientProvider(uri, conf);
+            KeyProvider kp = createProvider(uri, conf);
             try {
               KeyProvider.KeyVersion kv = kp.rollNewVersion("k1");
               Assert.assertNull(kv.getMaterial());
@@ -1280,7 +1292,7 @@ public class TestKMS {
         doAs("SET_KEY_MATERIAL", new PrivilegedExceptionAction<Void>() {
           @Override
           public Void run() throws Exception {
-            KeyProvider kp = new KMSClientProvider(uri, conf);
+            KeyProvider kp = createProvider(uri, conf);
             try {
               KeyProvider.KeyVersion kv =
                   kp.rollNewVersion("k1", new byte[16]);
@@ -1296,7 +1308,7 @@ public class TestKMS {
             doAs("GET", new PrivilegedExceptionAction<KeyVersion>() {
           @Override
           public KeyVersion run() throws Exception {
-            KeyProvider kp = new KMSClientProvider(uri, conf);
+            KeyProvider kp = createProvider(uri, conf);
             try {
               kp.getKeyVersion("k1@0");
               KeyVersion kv = kp.getCurrentKey("k1");
@@ -1313,7 +1325,7 @@ public class TestKMS {
                 new PrivilegedExceptionAction<EncryptedKeyVersion>() {
           @Override
           public EncryptedKeyVersion run() throws Exception {
-            KeyProvider kp = new KMSClientProvider(uri, conf);
+            KeyProvider kp = createProvider(uri, conf);
             try {
               KeyProviderCryptoExtension kpCE = KeyProviderCryptoExtension.
                       createKeyProviderCryptoExtension(kp);
@@ -1330,7 +1342,7 @@ public class TestKMS {
         doAs("DECRYPT_EEK", new PrivilegedExceptionAction<Void>() {
           @Override
           public Void run() throws Exception {
-            KeyProvider kp = new KMSClientProvider(uri, conf);
+            KeyProvider kp = createProvider(uri, conf);
             try {
               KeyProviderCryptoExtension kpCE = KeyProviderCryptoExtension.
                       createKeyProviderCryptoExtension(kp);
@@ -1345,7 +1357,7 @@ public class TestKMS {
         doAs("GET_KEYS", new PrivilegedExceptionAction<Void>() {
           @Override
           public Void run() throws Exception {
-            KeyProvider kp = new KMSClientProvider(uri, conf);
+            KeyProvider kp = createProvider(uri, conf);
             try {
               kp.getKeys();
             } catch (Exception ex) {
@@ -1358,7 +1370,7 @@ public class TestKMS {
         doAs("GET_METADATA", new PrivilegedExceptionAction<Void>() {
           @Override
           public Void run() throws Exception {
-            KeyProvider kp = new KMSClientProvider(uri, conf);
+            KeyProvider kp = createProvider(uri, conf);
             try {
               kp.getMetadata("k1");
               kp.getKeysMetadata("k1");
@@ -1385,7 +1397,7 @@ public class TestKMS {
           @Override
           public Void run() throws Exception {
             try {
-              KeyProvider kp = new KMSClientProvider(uri, conf);
+              KeyProvider kp = createProvider(uri, conf);
               KeyProvider.KeyVersion kv = kp.createKey("k2",
                   new KeyProvider.Options(conf));
               Assert.fail();
@@ -1440,12 +1452,12 @@ public class TestKMS {
           @Override
           public Void run() throws Exception {
             try {
-              KMSClientProvider kp = new KMSClientProvider(uri, conf);
+              KeyProvider kp = createProvider(uri, conf);
               KeyProvider.KeyVersion kv = kp.createKey("ck0",
                   new KeyProvider.Options(conf));
               EncryptedKeyVersion eek =
-                  kp.generateEncryptedKey("ck0");
-              kp.decryptEncryptedKey(eek);
+                  ((CryptoExtension)kp).generateEncryptedKey("ck0");
+              ((CryptoExtension)kp).decryptEncryptedKey(eek);
               Assert.assertNull(kv.getMaterial());
             } catch (Exception ex) {
               Assert.fail(ex.getMessage());
@@ -1458,12 +1470,12 @@ public class TestKMS {
           @Override
           public Void run() throws Exception {
             try {
-              KMSClientProvider kp = new KMSClientProvider(uri, conf);
+              KeyProvider kp = createProvider(uri, conf);
               KeyProvider.KeyVersion kv = kp.createKey("ck1",
                   new KeyProvider.Options(conf));
               EncryptedKeyVersion eek =
-                  kp.generateEncryptedKey("ck1");
-              kp.decryptEncryptedKey(eek);
+                  ((CryptoExtension)kp).generateEncryptedKey("ck1");
+              ((CryptoExtension)kp).decryptEncryptedKey(eek);
               Assert.fail("admin user must not be allowed to decrypt !!");
             } catch (Exception ex) {
             }
@@ -1475,12 +1487,12 @@ public class TestKMS {
           @Override
           public Void run() throws Exception {
             try {
-              KMSClientProvider kp = new KMSClientProvider(uri, conf);
+              KeyProvider kp = createProvider(uri, conf);
               KeyProvider.KeyVersion kv = kp.createKey("ck2",
                   new KeyProvider.Options(conf));
               EncryptedKeyVersion eek =
-                  kp.generateEncryptedKey("ck2");
-              kp.decryptEncryptedKey(eek);
+                  ((CryptoExtension)kp).generateEncryptedKey("ck2");
+              ((CryptoExtension)kp).decryptEncryptedKey(eek);
               Assert.fail("admin user must not be allowed to decrypt !!");
             } catch (Exception ex) {
             }
@@ -1525,7 +1537,7 @@ public class TestKMS {
           @Override
           public Void run() throws Exception {
             try {
-              KeyProvider kp = new KMSClientProvider(uri, conf);
+              KeyProvider kp = createProvider(uri, conf);
               KeyProvider.KeyVersion kv = kp.createKey("ck0",
                   new KeyProvider.Options(conf));
               Assert.assertNull(kv.getMaterial());
@@ -1540,7 +1552,7 @@ public class TestKMS {
           @Override
           public Void run() throws Exception {
             try {
-              KeyProvider kp = new KMSClientProvider(uri, conf);
+              KeyProvider kp = createProvider(uri, conf);
               KeyProvider.KeyVersion kv = kp.createKey("ck1",
                   new KeyProvider.Options(conf));
               Assert.assertNull(kv.getMaterial());
@@ -1583,7 +1595,7 @@ public class TestKMS {
 
     boolean caughtTimeout = false;
     try {
-      KeyProvider kp = new KMSClientProvider(uri, conf);
+      KeyProvider kp = createProvider(uri, conf);
       kp.getKeys();
     } catch (SocketTimeoutException e) {
       caughtTimeout = true;
@@ -1593,7 +1605,7 @@ public class TestKMS {
 
     caughtTimeout = false;
     try {
-      KeyProvider kp = new KMSClientProvider(uri, conf);
+      KeyProvider kp = createProvider(uri, conf);
       KeyProviderCryptoExtension.createKeyProviderCryptoExtension(kp)
           .generateEncryptedKey("a");
     } catch (SocketTimeoutException e) {
@@ -1604,7 +1616,7 @@ public class TestKMS {
 
     caughtTimeout = false;
     try {
-      KeyProvider kp = new KMSClientProvider(uri, conf);
+      KeyProvider kp = createProvider(uri, conf);
       KeyProviderCryptoExtension.createKeyProviderCryptoExtension(kp)
           .decryptEncryptedKey(
               new KMSClientProvider.KMSEncryptedKeyVersion("a",
@@ -1651,7 +1663,7 @@ public class TestKMS {
             UserGroupInformation.getCurrentUser();
 
         try {
-          KeyProvider kp = new KMSClientProvider(uri, conf);
+          KeyProvider kp = createProvider(uri, conf);
           kp.createKey(keyA, new KeyProvider.Options(conf));
         } catch (IOException ex) {
           System.out.println(ex.getMessage());
@@ -1660,7 +1672,7 @@ public class TestKMS {
         doAs("client", new PrivilegedExceptionAction<Void>() {
           @Override
           public Void run() throws Exception {
-            KeyProvider kp = new KMSClientProvider(uri, conf);
+            KeyProvider kp = createProvider(uri, conf);
             KeyProviderDelegationTokenExtension kpdte =
                 KeyProviderDelegationTokenExtension.
                     createKeyProviderDelegationTokenExtension(kp);
@@ -1672,7 +1684,7 @@ public class TestKMS {
         nonKerberosUgi.addCredentials(credentials);
 
         try {
-          KeyProvider kp = new KMSClientProvider(uri, conf);
+          KeyProvider kp = createProvider(uri, conf);
           kp.createKey(keyA, new KeyProvider.Options(conf));
         } catch (IOException ex) {
           System.out.println(ex.getMessage());
@@ -1681,7 +1693,7 @@ public class TestKMS {
         nonKerberosUgi.doAs(new PrivilegedExceptionAction<Void>() {
           @Override
           public Void run() throws Exception {
-            KeyProvider kp = new KMSClientProvider(uri, conf);
+            KeyProvider kp = createProvider(uri, conf);
             kp.createKey(keyD, new KeyProvider.Options(conf));
             return null;
           }
@@ -1767,7 +1779,7 @@ public class TestKMS {
                   new PrivilegedExceptionAction<KeyProvider>() {
                     @Override
                     public KeyProvider run() throws Exception {
-                      KMSClientProvider kp = new KMSClientProvider(uri, conf);
+                      KeyProvider kp = createProvider(uri, conf);
                           kp.createKey("k1", new byte[16],
                               new KeyProvider.Options(conf));
                           kp.createKey("k2", new byte[16],
@@ -1844,7 +1856,7 @@ public class TestKMS {
         clientUgi.doAs(new PrivilegedExceptionAction<Void>() {
           @Override
           public Void run() throws Exception {
-            final KeyProvider kp = new KMSClientProvider(uri, conf);
+            final KeyProvider kp = createProvider(uri, conf);
             kp.createKey("kaa", new KeyProvider.Options(conf));
 
             // authorized proxyuser
@@ -1956,7 +1968,7 @@ public class TestKMS {
             fooUgi.doAs(new PrivilegedExceptionAction<Void>() {
               @Override
               public Void run() throws Exception {
-                KeyProvider kp = new KMSClientProvider(uri, conf);
+                KeyProvider kp = createProvider(uri, conf);
                 Assert.assertNotNull(kp.createKey("kaa",
                     new KeyProvider.Options(conf)));
                 return null;
@@ -1970,7 +1982,7 @@ public class TestKMS {
               @Override
               public Void run() throws Exception {
                 try {
-                  KeyProvider kp = new KMSClientProvider(uri, conf);
+                  KeyProvider kp = createProvider(uri, conf);
                   kp.createKey("kbb", new KeyProvider.Options(conf));
                   Assert.fail();
                 } catch (Exception ex) {
@@ -1986,7 +1998,7 @@ public class TestKMS {
             barUgi.doAs(new PrivilegedExceptionAction<Void>() {
               @Override
               public Void run() throws Exception {
-                KeyProvider kp = new KMSClientProvider(uri, conf);
+                KeyProvider kp = createProvider(uri, conf);
                 Assert.assertNotNull(kp.createKey("kcc",
                     new KeyProvider.Options(conf)));
                 return null;


[37/50] [abbrv] hadoop git commit: YARN-3168. Convert site documentation from apt to markdown (Gururaj Shetty via aw)

Posted by zh...@apache.org.
YARN-3168. Convert site documentation from apt to markdown (Gururaj Shetty via aw)


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

Branch: refs/heads/HDFS-7285
Commit: 06aca7c61d037c506ad78420a0b88e562b1e4c2b
Parents: eaccaba
Author: Allen Wittenauer <aw...@apache.org>
Authored: Fri Feb 27 20:39:44 2015 -0800
Committer: Zhe Zhang <zh...@cloudera.com>
Committed: Mon Mar 2 09:13:54 2015 -0800

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                 |    3 +
 .../src/site/apt/CapacityScheduler.apt.vm       |  368 ---
 .../src/site/apt/DockerContainerExecutor.apt.vm |  204 --
 .../src/site/apt/FairScheduler.apt.vm           |  483 ---
 .../src/site/apt/NodeManager.apt.vm             |   64 -
 .../src/site/apt/NodeManagerCgroups.apt.vm      |   77 -
 .../src/site/apt/NodeManagerRest.apt.vm         |  645 ----
 .../src/site/apt/NodeManagerRestart.apt.vm      |   86 -
 .../src/site/apt/ResourceManagerHA.apt.vm       |  233 --
 .../src/site/apt/ResourceManagerRest.apt.vm     | 3104 ------------------
 .../src/site/apt/ResourceManagerRestart.apt.vm  |  298 --
 .../src/site/apt/SecureContainer.apt.vm         |  176 -
 .../src/site/apt/TimelineServer.apt.vm          |  260 --
 .../src/site/apt/WebApplicationProxy.apt.vm     |   49 -
 .../src/site/apt/WebServicesIntro.apt.vm        |  593 ----
 .../src/site/apt/WritingYarnApplications.apt.vm |  757 -----
 .../hadoop-yarn-site/src/site/apt/YARN.apt.vm   |   77 -
 .../src/site/apt/YarnCommands.apt.vm            |  369 ---
 .../hadoop-yarn-site/src/site/apt/index.apt.vm  |   82 -
 .../src/site/markdown/CapacityScheduler.md      |  186 ++
 .../site/markdown/DockerContainerExecutor.md.vm |  154 +
 .../src/site/markdown/FairScheduler.md          |  233 ++
 .../src/site/markdown/NodeManager.md            |   57 +
 .../src/site/markdown/NodeManagerCgroups.md     |   57 +
 .../src/site/markdown/NodeManagerRest.md        |  543 +++
 .../src/site/markdown/NodeManagerRestart.md     |   53 +
 .../src/site/markdown/ResourceManagerHA.md      |  140 +
 .../src/site/markdown/ResourceManagerRest.md    | 2640 +++++++++++++++
 .../src/site/markdown/ResourceManagerRestart.md |  181 +
 .../src/site/markdown/SecureContainer.md        |  135 +
 .../src/site/markdown/TimelineServer.md         |  231 ++
 .../src/site/markdown/WebApplicationProxy.md    |   24 +
 .../src/site/markdown/WebServicesIntro.md       |  569 ++++
 .../site/markdown/WritingYarnApplications.md    |  591 ++++
 .../hadoop-yarn-site/src/site/markdown/YARN.md  |   42 +
 .../src/site/markdown/YarnCommands.md           |  272 ++
 .../hadoop-yarn-site/src/site/markdown/index.md |   75 +
 37 files changed, 6186 insertions(+), 7925 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/06aca7c6/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index e7af84b..02b1831 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -20,6 +20,9 @@ Trunk - Unreleased
     YARN-2980. Move health check script related functionality to hadoop-common
     (Varun Saxena via aw)
 
+    YARN-3168. Convert site documentation from apt to markdown (Gururaj Shetty
+    via aw)
+
   OPTIMIZATIONS
 
   BUG FIXES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/06aca7c6/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/apt/CapacityScheduler.apt.vm
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/apt/CapacityScheduler.apt.vm b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/apt/CapacityScheduler.apt.vm
deleted file mode 100644
index 8528c1a..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/apt/CapacityScheduler.apt.vm
+++ /dev/null
@@ -1,368 +0,0 @@
-~~ 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 Map Reduce Next Generation-${project.version} - Capacity Scheduler
-  ---
-  ---
-  ${maven.build.timestamp}
-
-Hadoop MapReduce Next Generation - Capacity Scheduler
-
-%{toc|section=1|fromDepth=0}
-
-* {Purpose} 
-
-  This document describes the <<<CapacityScheduler>>>, a pluggable scheduler 
-  for Hadoop which allows for multiple-tenants to securely share a large cluster 
-  such that their applications are allocated resources in a timely manner under 
-  constraints of allocated capacities.
-
-* {Overview}
-
-  The <<<CapacityScheduler>>> is designed to run Hadoop applications as a 
-  shared, multi-tenant cluster in an operator-friendly manner while maximizing 
-  the throughput and the utilization of the cluster.
-   
-  Traditionally each organization has it own private set of compute resources 
-  that have sufficient capacity to meet the organization's SLA under peak or 
-  near peak conditions. This generally leads to poor average utilization and 
-  overhead of managing multiple independent clusters, one per each organization. 
-  Sharing clusters between organizations is a cost-effective manner of running 
-  large Hadoop installations since this allows them to reap benefits of
-  economies of scale without creating private clusters. However, organizations 
-  are concerned about sharing a cluster because they are worried about others 
-  using the resources that are critical for their SLAs. 
-   
-  The <<<CapacityScheduler>>> is designed to allow sharing a large cluster while 
-  giving each organization capacity guarantees. The central idea is 
-  that the available resources in the Hadoop cluster are shared among multiple 
-  organizations who collectively fund the cluster based on their computing 
-  needs. There is an added benefit that an organization can access 
-  any excess capacity not being used by others. This provides elasticity for 
-  the organizations in a cost-effective manner.
-   
-  Sharing clusters across organizations necessitates strong support for
-  multi-tenancy since each organization must be guaranteed capacity and 
-  safe-guards to ensure the shared cluster is impervious to single rouge 
-  application or user or sets thereof. The <<<CapacityScheduler>>> provides a 
-  stringent set of limits to ensure that a single application or user or queue 
-  cannot consume disproportionate amount of resources in the cluster. Also, the 
-  <<<CapacityScheduler>>> provides limits on initialized/pending applications 
-  from a single user and queue to ensure fairness and stability of the cluster.
-   
-  The primary abstraction provided by the <<<CapacityScheduler>>> is the concept 
-  of <queues>. These queues are typically setup by administrators to reflect the 
-  economics of the shared cluster. 
-  
-  To provide further control and predictability on sharing of resources, the 
-  <<<CapacityScheduler>>> supports <hierarchical queues> to ensure 
-  resources are shared among the sub-queues of an organization before other 
-  queues are allowed to use free resources, there-by providing <affinity> 
-  for sharing free resources among applications of a given organization.
-   
-* {Features}
-
-  The <<<CapacityScheduler>>> supports the following features:
-  
-  * Hierarchical Queues - Hierarchy of queues is supported to ensure resources 
-    are shared among the sub-queues of an organization before other 
-    queues are allowed to use free resources, there-by providing more control
-    and predictability.
-    
-  * Capacity Guarantees - Queues are allocated a fraction of the capacity of the 
-    grid in the sense that a certain capacity of resources will be at their 
-    disposal. All applications submitted to a queue will have access to the 
-    capacity allocated to the queue. Adminstrators can configure soft limits and 
-    optional hard limits on the capacity allocated to each queue.
-    
-  * Security - Each queue has strict ACLs which controls which users can submit 
-    applications to individual queues. Also, there are safe-guards to ensure 
-    that users cannot view and/or modify applications from other users.
-    Also, per-queue and system administrator roles are supported.
-    
-  * Elasticity - Free resources can be allocated to any queue beyond it's 
-    capacity. When there is demand for these resources from queues running below 
-    capacity at a future point in time, as tasks scheduled on these resources 
-    complete, they will be assigned to applications on queues running below the
-    capacity (pre-emption is not supported). This ensures that resources are available 
-    in a predictable and elastic manner to queues, thus preventing artifical silos 
-    of resources in the cluster which helps utilization.
-    
-  * Multi-tenancy - Comprehensive set of limits are provided to prevent a 
-    single application, user and queue from monopolizing resources of the queue 
-    or the cluster as a whole to ensure that the cluster isn't overwhelmed.
-    
-  * Operability
-  
-    * Runtime Configuration - The queue definitions and properties such as 
-      capacity, ACLs can be changed, at runtime, by administrators in a secure 
-      manner to minimize disruption to users. Also, a console is provided for 
-      users and administrators to view current allocation of resources to 
-      various queues in the system. Administrators can <add additional queues> 
-      at runtime, but queues cannot be <deleted> at runtime.
-      
-    * Drain applications - Administrators can <stop> queues
-      at runtime to ensure that while existing applications run to completion,
-      no new applications can be submitted. If a queue is in <<<STOPPED>>> 
-      state, new applications cannot be submitted to <itself> or 
-      <any of its child queueus>. Existing applications continue to completion, 
-      thus the queue can be <drained> gracefully.  Administrators can also 
-      <start> the stopped queues. 
-    
-  * Resource-based Scheduling - Support for resource-intensive applications, 
-    where-in a application can optionally specify higher resource-requirements 
-    than the default, there-by accomodating applications with differing resource
-    requirements. Currently, <memory> is the the resource requirement supported.
-  
-  []
-  
-* {Configuration}
-
-  * Setting up <<<ResourceManager>>> to use <<<CapacityScheduler>>>
-  
-    To configure the <<<ResourceManager>>> to use the <<<CapacityScheduler>>>, set
-    the following property in the <<conf/yarn-site.xml>>:
-  
-*--------------------------------------+--------------------------------------+
-|| Property                            || Value                                |
-*--------------------------------------+--------------------------------------+
-| <<<yarn.resourcemanager.scheduler.class>>> | |
-| | <<<org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler>>> |
-*--------------------------------------+--------------------------------------+
-
-  * Setting up <queues>
-   
-    <<conf/capacity-scheduler.xml>> is the configuration file for the
-    <<<CapacityScheduler>>>.  
-  
-    The <<<CapacityScheduler>>> has a pre-defined queue called <root>. All 
-    queueus in the system are children of the root queue.
-
-    Further queues can be setup by configuring 
-    <<<yarn.scheduler.capacity.root.queues>>> with a list of comma-separated
-    child queues.
-    
-    The configuration for <<<CapacityScheduler>>> uses a concept called
-    <queue path> to configure the hierarchy of queues. The <queue path> is the
-    full path of the queue's hierarchy, starting at <root>, with . (dot) as the 
-    delimiter.
-    
-    A given queue's children can be defined with the configuration knob:
-    <<<yarn.scheduler.capacity.<queue-path>.queues>>>. Children do not 
-    inherit properties directly from the parent unless otherwise noted.
-
-    Here is an example with three top-level child-queues <<<a>>>, <<<b>>> and 
-    <<<c>>> and some sub-queues for <<<a>>> and <<<b>>>:
-     
-----    
-<property>
-  <name>yarn.scheduler.capacity.root.queues</name>
-  <value>a,b,c</value>
-  <description>The queues at the this level (root is the root queue).
-  </description>
-</property>
-
-<property>
-  <name>yarn.scheduler.capacity.root.a.queues</name>
-  <value>a1,a2</value>
-  <description>The queues at the this level (root is the root queue).
-  </description>
-</property>
-
-<property>
-  <name>yarn.scheduler.capacity.root.b.queues</name>
-  <value>b1,b2,b3</value>
-  <description>The queues at the this level (root is the root queue).
-  </description>
-</property>
-----    
-
-  * Queue Properties
-  
-    * Resource Allocation
-  
-*--------------------------------------+--------------------------------------+
-|| Property                            || Description                         |
-*--------------------------------------+--------------------------------------+
-| <<<yarn.scheduler.capacity.<queue-path>.capacity>>> | |
-| | Queue <capacity> in percentage (%) as a float (e.g. 12.5).| 
-| | The sum of capacities for all queues, at each level, must be equal |
-| | to 100. | 
-| | Applications in the queue may consume more resources than the queue's | 
-| | capacity if there are free resources, providing elasticity. |
-*--------------------------------------+--------------------------------------+
-| <<<yarn.scheduler.capacity.<queue-path>.maximum-capacity>>> |   | 
-| | Maximum queue capacity in percentage (%) as a float. |
-| | This limits the <elasticity> for applications in the queue. |
-| | Defaults to -1 which disables it. |
-*--------------------------------------+--------------------------------------+
-| <<<yarn.scheduler.capacity.<queue-path>.minimum-user-limit-percent>>> |   | 
-| | Each queue enforces a limit on the percentage of resources allocated to a | 
-| | user at any given time, if there is demand for resources. The user limit | 
-| | can vary between a minimum and maximum value. The the former |
-| | (the minimum value) is set to this property value and the latter |
-| | (the maximum value) depends on the number of users who have submitted |
-| | applications. For e.g., suppose the value of this property is 25. | 
-| | If two users have submitted applications to a queue, no single user can |
-| | use more than 50% of the queue resources. If a third user submits an | 
-| | application, no single user can use more than 33% of the queue resources. |
-| | With 4 or more users, no user can use more than 25% of the queues |
-| | resources. A value of 100 implies no user limits are imposed. The default |
-| | is 100. Value is specified as a integer.|
-*--------------------------------------+--------------------------------------+
-| <<<yarn.scheduler.capacity.<queue-path>.user-limit-factor>>> |   | 
-| | The multiple of the queue capacity which can be configured to allow a | 
-| | single user to acquire more resources. By default this is set to 1 which | 
-| | ensures that a single user can never take more than the queue's configured | 
-| | capacity irrespective of how idle th cluster is. Value is specified as |
-| | a float.|
-*--------------------------------------+--------------------------------------+
-| <<<yarn.scheduler.capacity.<queue-path>.maximum-allocation-mb>>> |   |
-| | The per queue maximum limit of memory to allocate to each container |
-| | request at the Resource Manager. This setting overrides the cluster |
-| | configuration <<<yarn.scheduler.maximum-allocation-mb>>>. This value |
-| | must be smaller than or equal to the cluster maximum. |
-*--------------------------------------+--------------------------------------+
-| <<<yarn.scheduler.capacity.<queue-path>.maximum-allocation-vcores>>> |   |
-| | The per queue maximum limit of virtual cores to allocate to each container |
-| | request at the Resource Manager. This setting overrides the cluster |
-| | configuration <<<yarn.scheduler.maximum-allocation-vcores>>>. This value |
-| | must be smaller than or equal to the cluster maximum. |
-*--------------------------------------+--------------------------------------+
-
-    * Running and Pending Application Limits
-    
-    
-    The <<<CapacityScheduler>>> supports the following parameters to control 
-    the running and pending applications:
-    
-
-*--------------------------------------+--------------------------------------+
-|| Property                            || Description                         |
-*--------------------------------------+--------------------------------------+
-| <<<yarn.scheduler.capacity.maximum-applications>>> /  |
-| <<<yarn.scheduler.capacity.<queue-path>.maximum-applications>>>  | |
-| | Maximum number of applications in the system which can be concurrently |
-| | active both running and pending. Limits on each queue are directly |
-| | proportional to their queue capacities and user limits. This is a 
-| | hard limit and any applications submitted when this limit is reached will |
-| | be rejected. Default is 10000. This can be set for all queues with |
-| | <<<yarn.scheduler.capacity.maximum-applications>>> and can also be overridden on a  |
-| | per queue basis by setting <<<yarn.scheduler.capacity.<queue-path>.maximum-applications>>>. |
-| | Integer value expected.|
-*--------------------------------------+--------------------------------------+
-| <<<yarn.scheduler.capacity.maximum-am-resource-percent>>> / |
-| <<<yarn.scheduler.capacity.<queue-path>.maximum-am-resource-percent>>> | |
-| | Maximum percent of resources in the cluster which can be used to run |
-| | application masters - controls number of concurrent active applications. Limits on each |
-| | queue are directly proportional to their queue capacities and user limits. |
-| | Specified as a float - ie 0.5 = 50%. Default is 10%. This can be set for all queues with |
-| | <<<yarn.scheduler.capacity.maximum-am-resource-percent>>> and can also be overridden on a  |
-| | per queue basis by setting <<<yarn.scheduler.capacity.<queue-path>.maximum-am-resource-percent>>> |
-*--------------------------------------+--------------------------------------+
-
-    * Queue Administration & Permissions
-    
-    The <<<CapacityScheduler>>> supports the following parameters to  
-    the administer the queues:
-    
-    
-*--------------------------------------+--------------------------------------+
-|| Property                            || Description                         |
-*--------------------------------------+--------------------------------------+
-| <<<yarn.scheduler.capacity.<queue-path>.state>>> | |
-| | The <state> of the queue. Can be one of <<<RUNNING>>> or <<<STOPPED>>>. |
-| | If a queue is in <<<STOPPED>>> state, new applications cannot be |
-| | submitted to <itself> or <any of its child queues>. | 
-| | Thus, if the <root> queue is <<<STOPPED>>> no applications can be | 
-| | submitted to the entire cluster. |
-| | Existing applications continue to completion, thus the queue can be 
-| | <drained> gracefully. Value is specified as Enumeration. |
-*--------------------------------------+--------------------------------------+
-| <<<yarn.scheduler.capacity.root.<queue-path>.acl_submit_applications>>> | |
-| | The <ACL> which controls who can <submit> applications to the given queue. |
-| | If the given user/group has necessary ACLs on the given queue or |
-| | <one of the parent queues in the hierarchy> they can submit applications. |
-| | <ACLs> for this property <are> inherited from the parent queue |
-| | if not specified. |
-*--------------------------------------+--------------------------------------+
-| <<<yarn.scheduler.capacity.root.<queue-path>.acl_administer_queue>>> | |
-| | The <ACL> which controls who can <administer> applications on the given queue. |
-| | If the given user/group has necessary ACLs on the given queue or |
-| | <one of the parent queues in the hierarchy> they can administer applications. |
-| | <ACLs> for this property <are> inherited from the parent queue |
-| | if not specified. |
-*--------------------------------------+--------------------------------------+
-    
-    <Note:> An <ACL> is of the form <user1>, <user2><space><group1>, <group2>.
-    The special value of <<*>> implies <anyone>. The special value of <space>
-    implies <no one>. The default is <<*>> for the root queue if not specified.
-
-  * Other Properties
-
-    * Resource Calculator
-
-
-*--------------------------------------+--------------------------------------+
-|| Property                            || Description                         |
-*--------------------------------------+--------------------------------------+
-| <<<yarn.scheduler.capacity.resource-calculator>>> | |
-| | The ResourceCalculator implementation to be used to compare Resources in the |
-| | scheduler. The default i.e. org.apache.hadoop.yarn.util.resource.DefaultResourseCalculator |
-| | only uses Memory while DominantResourceCalculator uses Dominant-resource |
-| | to compare multi-dimensional resources such as Memory, CPU etc. A Java |
-| | ResourceCalculator class name is expected. |
-*--------------------------------------+--------------------------------------+
-
-
-    * Data Locality
-
-*--------------------------------------+--------------------------------------+
-|| Property                            || Description                         |
-*--------------------------------------+--------------------------------------+
-| <<<yarn.scheduler.capacity.node-locality-delay>>> | |
-| | Number of missed scheduling opportunities after which the CapacityScheduler |
-| | attempts to schedule rack-local containers. Typically, this should be set to |
-| | number of nodes in the cluster. By default is setting approximately number |
-| | of nodes in one rack which is 40. Positive integer value is expected.|
-*--------------------------------------+--------------------------------------+
-
-
-  * Reviewing the configuration of the CapacityScheduler
-
-      Once the installation and configuration is completed, you can review it 
-      after starting the YARN cluster from the web-ui.
-
-    * Start the YARN cluster in the normal manner.
-
-    * Open the <<<ResourceManager>>> web UI.
-
-    * The </scheduler> web-page should show the resource usages of individual 
-      queues.
-      
-      []
-      
-* {Changing Queue Configuration}
-
-  Changing queue properties and adding new queues is very simple. You need to
-  edit <<conf/capacity-scheduler.xml>> and run <yarn rmadmin -refreshQueues>.
-  
-----
-$ vi $HADOOP_CONF_DIR/capacity-scheduler.xml
-$ $HADOOP_YARN_HOME/bin/yarn rmadmin -refreshQueues
-----  
-
-  <Note:> Queues cannot be <deleted>, only addition of new queues is supported -
-  the updated queue configuration should be a valid one i.e. queue-capacity at
-  each <level> should be equal to 100%.
-

http://git-wip-us.apache.org/repos/asf/hadoop/blob/06aca7c6/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/apt/DockerContainerExecutor.apt.vm
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/apt/DockerContainerExecutor.apt.vm b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/apt/DockerContainerExecutor.apt.vm
deleted file mode 100644
index db75de9..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/apt/DockerContainerExecutor.apt.vm
+++ /dev/null
@@ -1,204 +0,0 @@
-
-~~ 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 Map Reduce Next Generation-${project.version} - Docker Container Executor
-  ---
-  ---
-  ${maven.build.timestamp}
-
-Docker Container Executor
-
-%{toc|section=1|fromDepth=0}
-
-* {Overview}
-
-    Docker (https://www.docker.io/) combines an easy-to-use interface to
-Linux containers with easy-to-construct image files for those
-containers.  In short, Docker launches very light weight virtual
-machines.
-
-    The Docker Container Executor (DCE) allows the YARN NodeManager to
-launch YARN containers into Docker containers.  Users can specify the
-Docker images they want for their YARN containers.  These containers
-provide a custom software environment in which the user's code runs,
-isolated from the software environment of the NodeManager.  These
-containers can include special libraries needed by the application,
-and they can have different versions of Perl, Python, and even Java
-than what is installed on the NodeManager.  Indeed, these containers
-can run a different flavor of Linux than what is running on the
-NodeManager -- although the YARN container must define all the environments
- and libraries needed to run the job, nothing will be shared with the NodeManager.
-
-   Docker for YARN provides both consistency (all YARN containers will
-have the same software environment) and isolation (no interference
-with whatever is installed on the physical machine).
-  
-* {Cluster Configuration}
-
-    Docker Container Executor runs in non-secure mode of HDFS and
-YARN. It will not run in secure mode, and will exit if it detects
-secure mode.
-
-    The DockerContainerExecutor requires Docker daemon to be running on
-the NodeManagers, and the Docker client installed and able to start Docker
-containers.  To prevent timeouts while starting jobs, the Docker
-images to be used by a job should already be downloaded in the
-NodeManagers. Here's an example of how this can be done:
-
-----
-sudo docker pull sequenceiq/hadoop-docker:2.4.1
-----
-
-   This should be done as part of the NodeManager startup.
-
-   The following properties must be set in yarn-site.xml:
-
-----
-<property>
- <name>yarn.nodemanager.docker-container-executor.exec-name</name>
-  <value>/usr/bin/docker</value>
-  <description>
-     Name or path to the Docker client. This is a required parameter. If this is empty,
-     user must pass an image name as part of the job invocation(see below).
-  </description>
-</property>
-
-<property>
-  <name>yarn.nodemanager.container-executor.class</name>
-  <value>org.apache.hadoop.yarn.server.nodemanager.DockerContainerExecutor</value>
-  <description>
-     This is the container executor setting that ensures that all
-jobs are started with the DockerContainerExecutor.
-  </description>
-</property>
-----
-
-   Administrators should be aware that DCE doesn't currently provide
-user name-space isolation.  This means, in particular, that software
-running as root in the YARN container will have root privileges in the
-underlying NodeManager.  Put differently, DCE currently provides no
-better security guarantees than YARN's Default Container Executor. In
-fact, DockerContainerExecutor will exit if it detects secure yarn.
-
-* {Tips for connecting to a secure docker repository}
-
-   By default, docker images are pulled from the docker public repository. The
-format of a docker image url is: <username>/<image_name>. For example,
-sequenceiq/hadoop-docker:2.4.1 is an image in docker public repository that contains java and
-hadoop.
-
-   If you want your own private repository, you provide the repository url instead of
-your username. Therefore, the image url becomes: <private_repo_url>/<image_name>.
-For example, if your repository is on localhost:8080, your images would be like:
- localhost:8080/hadoop-docker
-
-   To connect to a secure docker repository, you can use the following invocation:
-
-----
-docker login [OPTIONS] [SERVER]
-
-Register or log in to a Docker registry server, if no server is specified
-"https://index.docker.io/v1/" is the default.
-
--e, --email=""       Email
--p, --password=""    Password
--u, --username=""    Username
-----
-
-   If you want to login to a self-hosted registry you can specify this by adding
-the server name.
-
-----
-docker login <private_repo_url>
-----
-
-   This needs to be run as part of the NodeManager startup, or as a cron job if
-the login session expires periodically. You can login to multiple docker repositories
-from the same NodeManager, but all your users will have access to all your repositories,
-as at present the DockerContainerExecutor does not support per-job docker login.
-
-* {Job Configuration}
-
-   Currently you cannot configure any of the Docker settings with the job configuration.
-You can provide Mapper, Reducer, and ApplicationMaster environment overrides for the
-docker images, using the following 3 JVM properties respectively(only for MR jobs):
-
-  * mapreduce.map.env: You can override the mapper's image by passing
-    yarn.nodemanager.docker-container-executor.image-name=<your_image_name>
-    to this JVM property.
-
-  * mapreduce.reduce.env: You can override the reducer's image by passing
-    yarn.nodemanager.docker-container-executor.image-name=<your_image_name>
-    to this JVM property.
-
-  * yarn.app.mapreduce.am.env: You can override the ApplicationMaster's image
-    by passing yarn.nodemanager.docker-container-executor.image-name=<your_image_name>
-    to this JVM property.
-
-* {Docker Image requirements}
-
-   The Docker Images used for YARN containers must meet the following
-requirements:
-
-   The distro and version of Linux in your Docker Image can be quite different 
-from that of your NodeManager.  (Docker does have a few limitations in this 
-regard, but you're not likely to hit them.)  However, if you're using the 
-MapReduce framework, then your image will need to be configured for running 
-Hadoop. Java must be installed in the container, and the following environment variables
-must be defined in the image: JAVA_HOME, HADOOP_COMMON_PATH, HADOOP_HDFS_HOME,
-HADOOP_MAPRED_HOME, HADOOP_YARN_HOME, and HADOOP_CONF_DIR
-
-
-* {Working example of yarn launched docker containers.}
-
-  The following example shows how to run teragen using DockerContainerExecutor.
-
-  * First ensure that YARN is properly configured with DockerContainerExecutor(see above).
-
-----
-<property>
- <name>yarn.nodemanager.docker-container-executor.exec-name</name>
-  <value>docker -H=tcp://0.0.0.0:4243</value>
-  <description>
-     Name or path to the Docker client. The tcp socket must be
-     where docker daemon is listening.
-  </description>
-</property>
-
-<property>
-  <name>yarn.nodemanager.container-executor.class</name>
-  <value>org.apache.hadoop.yarn.server.nodemanager.DockerContainerExecutor</value>
-  <description>
-     This is the container executor setting that ensures that all
-jobs are started with the DockerContainerExecutor.
-  </description>
-</property>
-----
-
-  * Pick a custom Docker image if you want. In this example, we'll use sequenceiq/hadoop-docker:2.4.1 from the
-docker hub repository. It has jdk, hadoop, and all the previously mentioned environment variables configured.
-
-  * Run:
-
-----
-hadoop jar $HADOOP_INSTALLATION_DIR/share/hadoop/mapreduce/hadoop-mapreduce-examples-*.jar \
-teragen \
--Dmapreduce.map.env="yarn.nodemanager.docker-container-executor.image-name=sequenceiq/hadoop-docker:2.4.1" \
--Dyarn.app.mapreduce.am.env="yarn.nodemanager.docker-container-executor.image-name=sequenceiq/hadoop-docker:2.4.1" \
-1000 \
-teragen_out_dir
-----
-
-  Once it succeeds, you can check the yarn debug logs to verify that docker indeed has launched containers.
-

http://git-wip-us.apache.org/repos/asf/hadoop/blob/06aca7c6/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/apt/FairScheduler.apt.vm
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/apt/FairScheduler.apt.vm b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/apt/FairScheduler.apt.vm
deleted file mode 100644
index 10de3e0..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/apt/FairScheduler.apt.vm
+++ /dev/null
@@ -1,483 +0,0 @@
-~~ 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 Map Reduce Next Generation-${project.version} - Fair Scheduler
-  ---
-  ---
-  ${maven.build.timestamp}
-
-Hadoop MapReduce Next Generation - Fair Scheduler
-
-%{toc|section=1|fromDepth=0}
-
-* {Purpose} 
-
-  This document describes the <<<FairScheduler>>>, a pluggable scheduler for Hadoop 
-  that allows YARN applications to share resources in large clusters fairly.
-
-* {Introduction}
-
-  Fair scheduling is a method of assigning resources to applications such that 
-  all apps get, on average, an equal share of resources over time.
-  Hadoop NextGen is capable of scheduling multiple resource types. By default,
-  the Fair Scheduler bases scheduling fairness decisions only on memory. It
-  can be configured to schedule with both memory and CPU, using the notion
-  of Dominant Resource Fairness developed by Ghodsi et al. When there is a
-  single app running, that app uses the entire cluster. When other apps are
-  submitted, resources that free up are assigned to the new apps, so that each
-  app eventually on gets roughly the same amount of resources. Unlike the default
-  Hadoop scheduler, which forms a queue of apps, this lets short apps finish in
-  reasonable time while not starving long-lived apps. It is also a reasonable way
-  to share a cluster between a number of users. Finally, fair sharing can also
-  work with app priorities - the priorities are used as weights to determine the 
-  fraction of total resources that each app should get.
-
-  The scheduler organizes apps further into "queues", and shares resources
-  fairly between these queues. By default, all users share a single queue,
-  named "default". If an app specifically lists a queue in a container resource
-  request, the request is submitted to that queue. It is also possible to assign
-  queues based on the user name included with the request through
-  configuration. Within each queue, a scheduling policy is used to share
-  resources between the running apps. The default is memory-based fair sharing,
-  but FIFO and multi-resource with Dominant Resource Fairness can also be
-  configured. Queues can be arranged in a hierarchy to divide resources and
-  configured with weights to share the cluster in specific proportions.
-
-  In addition to providing fair sharing, the Fair Scheduler allows assigning 
-  guaranteed minimum shares to queues, which is useful for ensuring that 
-  certain users, groups or production applications always get sufficient 
-  resources. When a queue contains apps, it gets at least its minimum share, 
-  but when the queue does not need its full guaranteed share, the excess is 
-  split between other running apps. This lets the scheduler guarantee capacity 
-  for queues while utilizing resources efficiently when these queues don't
-  contain applications.
-
-  The Fair Scheduler lets all apps run by default, but it is also possible to 
-  limit the number of running apps per user and per queue through the config 
-  file. This can be useful when a user must submit hundreds of apps at once, 
-  or in general to improve performance if running too many apps at once would 
-  cause too much intermediate data to be created or too much context-switching.
-  Limiting the apps does not cause any subsequently submitted apps to fail, 
-  only to wait in the scheduler's queue until some of the user's earlier apps 
-  finish. 
-
-* {Hierarchical queues with pluggable policies}
-
-  The fair scheduler supports hierarchical queues. All queues descend from a
-  queue named "root". Available resources are distributed among the children
-  of the root queue in the typical fair scheduling fashion. Then, the children
-  distribute the resources assigned to them to their children in the same
-  fashion.  Applications may only be scheduled on leaf queues. Queues can be
-  specified as children of other queues by placing them as sub-elements of 
-  their parents in the fair scheduler allocation file.
-  
-  A queue's name starts with the names of its parents, with periods as
-  separators. So a queue named "queue1" under the root queue, would be referred
-  to as "root.queue1", and a queue named "queue2" under a queue named "parent1"
-  would be referred to as "root.parent1.queue2". When referring to queues, the
-  root part of the name is optional, so queue1 could be referred to as just
-  "queue1", and a queue2 could be referred to as just "parent1.queue2".
-
-  Additionally, the fair scheduler allows setting a different custom policy for
-  each queue to allow sharing the queue's resources in any which way the user
-  wants. A custom policy can be built by extending
-  <<<org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.SchedulingPolicy>>>.
-  FifoPolicy, FairSharePolicy (default), and DominantResourceFairnessPolicy are
-  built-in and can be readily used.
-
-  Certain add-ons are not yet supported which existed in the original (MR1) 
-  Fair Scheduler. Among them, is the use of a custom policies governing 
-  priority "boosting" over  certain apps. 
-
-* {Automatically placing applications in queues}
-
-  The Fair Scheduler allows administrators to configure policies that
-  automatically place submitted applications into appropriate queues. Placement
-  can depend on the user and groups of the submitter and the requested queue
-  passed by the application. A policy consists of a set of rules that are applied
-  sequentially to classify an incoming application. Each rule either places the
-  app into a queue, rejects it, or continues on to the next rule. Refer to the
-  allocation file format below for how to configure these policies.
-
-* {Installation}
-
-  To use the Fair Scheduler first assign the appropriate scheduler class in 
-  yarn-site.xml:
-
-------
-<property>
-  <name>yarn.resourcemanager.scheduler.class</name>
-  <value>org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.FairScheduler</value>
-</property>
-------
-
-* {Configuration}
-
-  Customizing the Fair Scheduler typically involves altering two files. First, 
-  scheduler-wide options can be set by adding configuration properties in the 
-  yarn-site.xml file in your existing configuration directory. Second, in 
-  most cases users will want to create an allocation file listing which queues 
-  exist and their respective weights and capacities. The allocation file
-  is reloaded every 10 seconds, allowing changes to be made on the fly.
-
-Properties that can be placed in yarn-site.xml
-
- * <<<yarn.scheduler.fair.allocation.file>>>
-
-   * Path to allocation file. An allocation file is an XML manifest describing
-     queues and their properties, in addition to certain policy defaults. This file
-     must be in the XML format described in the next section. If a relative path is
-     given, the file is searched for on the classpath (which typically includes
-     the Hadoop conf directory).
-     Defaults to fair-scheduler.xml.
-
- * <<<yarn.scheduler.fair.user-as-default-queue>>>
-
-    * Whether to use the username associated with the allocation as the default 
-      queue name, in the event that a queue name is not specified. If this is set 
-      to "false" or unset, all jobs have a shared default queue, named "default".
-      Defaults to true.  If a queue placement policy is given in the allocations
-      file, this property is ignored.
-
- * <<<yarn.scheduler.fair.preemption>>>
-
-    * Whether to use preemption. Defaults to false.
-
- * <<<yarn.scheduler.fair.preemption.cluster-utilization-threshold>>>
-
-    * The utilization threshold after which preemption kicks in. The
-      utilization is computed as the maximum ratio of usage to capacity among
-      all resources. Defaults to 0.8f.
-
- * <<<yarn.scheduler.fair.sizebasedweight>>>
-  
-    * Whether to assign shares to individual apps based on their size, rather than
-      providing an equal share to all apps regardless of size. When set to true,
-      apps are weighted by the natural logarithm of one plus the app's total
-      requested memory, divided by the natural logarithm of 2. Defaults to false.
-
- * <<<yarn.scheduler.fair.assignmultiple>>>
-
-    * Whether to allow multiple container assignments in one heartbeat. Defaults
-      to false.
-
- * <<<yarn.scheduler.fair.max.assign>>>
-
-    * If assignmultiple is true, the maximum amount of containers that can be
-      assigned in one heartbeat. Defaults to -1, which sets no limit.
-
- * <<<yarn.scheduler.fair.locality.threshold.node>>>
-
-    * For applications that request containers on particular nodes, the number of
-      scheduling opportunities since the last container assignment to wait before
-      accepting a placement on another node. Expressed as a float between 0 and 1,
-      which, as a fraction of the cluster size, is the number of scheduling
-      opportunities to pass up. The default value of -1.0 means don't pass up any
-      scheduling opportunities.
-
- * <<<yarn.scheduler.fair.locality.threshold.rack>>>
-
-    * For applications that request containers on particular racks, the number of
-      scheduling opportunities since the last container assignment to wait before
-      accepting a placement on another rack. Expressed as a float between 0 and 1,
-      which, as a fraction of the cluster size, is the number of scheduling
-      opportunities to pass up. The default value of -1.0 means don't pass up any
-      scheduling opportunities.
-
- * <<<yarn.scheduler.fair.allow-undeclared-pools>>>
-
-    * If this is true, new queues can be created at application submission time,
-      whether because they are specified as the application's queue by the
-      submitter or because they are placed there by the user-as-default-queue
-      property. If this is false, any time an app would be placed in a queue that
-      is not specified in the allocations file, it is placed in the "default" queue
-      instead. Defaults to true. If a queue placement policy is given in the
-      allocations file, this property is ignored.
-
- * <<<yarn.scheduler.fair.update-interval-ms>>>
- 
-    * The interval at which to lock the scheduler and recalculate fair shares,
-      recalculate demand, and check whether anything is due for preemption.
-      Defaults to 500 ms. 
-
-Allocation file format
-
-  The allocation file must be in XML format. The format contains five types of
-  elements:
-
- * <<Queue elements>>, which represent queues. Queue elements can take an optional
-   attribute 'type', which when set to 'parent' makes it a parent queue. This is useful
-   when we want to create a parent queue without configuring any leaf queues.
-   Each queue element may contain the following properties:
-
-   * minResources: minimum resources the queue is entitled to, in the form
-     "X mb, Y vcores". For the single-resource fairness policy, the vcores
-     value is ignored. If a queue's minimum share is not satisfied, it will be
-     offered available resources before any other queue under the same parent.
-     Under the single-resource fairness policy, a queue
-     is considered unsatisfied if its memory usage is below its minimum memory
-     share. Under dominant resource fairness, a queue is considered unsatisfied
-     if its usage for its dominant resource with respect to the cluster capacity
-     is below its minimum share for that resource. If multiple queues are
-     unsatisfied in this situation, resources go to the queue with the smallest
-     ratio between relevant resource usage and minimum. Note that it is
-     possible that a queue that is below its minimum may not immediately get up
-     to its minimum when it submits an application, because already-running jobs
-     may be using those resources.
-
-   * maxResources: maximum resources a queue is allowed, in the form
-     "X mb, Y vcores". For the single-resource fairness policy, the vcores
-     value is ignored. A queue will never be assigned a container that would
-     put its aggregate usage over this limit.
-
-   * maxRunningApps: limit the number of apps from the queue to run at once
-
-   * maxAMShare: limit the fraction of the queue's fair share that can be used
-     to run application masters. This property can only be used for leaf queues.
-     For example, if set to 1.0f, then AMs in the leaf queue can take up to 100%
-     of both the memory and CPU fair share. The value of -1.0f will disable
-     this feature and the amShare will not be checked. The default value is 0.5f.
-
-   * weight: to share the cluster non-proportionally with other queues. Weights
-     default to 1, and a queue with weight 2 should receive approximately twice
-     as many resources as a queue with the default weight.
-
-   * schedulingPolicy: to set the scheduling policy of any queue. The allowed
-     values are "fifo"/"fair"/"drf" or any class that extends
-     <<<org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.SchedulingPolicy>>>. 
-     Defaults to "fair". If "fifo", apps with earlier submit times are given preference
-     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 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.
-     If not set, the queue will inherit the value from its parent queue.
-
-   * fairSharePreemptionTimeout: number of seconds the queue is under its fair share
-     threshold before it will try to preempt containers to take resources from other
-     queues. If not set, the queue will inherit the value from its parent queue.
-
-   * fairSharePreemptionThreshold: the fair share preemption threshold for the
-     queue. If the queue waits fairSharePreemptionTimeout without receiving
-     fairSharePreemptionThreshold*fairShare resources, it is allowed to preempt
-     containers to take resources from other queues. If not set, the queue will
-     inherit the value from its parent queue.
-
- * <<User elements>>, which represent settings governing the behavior of individual 
-     users. They can contain a single property: maxRunningApps, a limit on the 
-     number of running apps for a particular user.
-
- * <<A userMaxAppsDefault element>>, which sets the default running app limit 
-   for any users whose limit is not otherwise specified.
-
- * <<A defaultFairSharePreemptionTimeout element>>, which sets the fair share
-   preemption timeout for the root queue; overridden by fairSharePreemptionTimeout
-   element in root queue.
-
- * <<A defaultMinSharePreemptionTimeout element>>, which sets the min share
-   preemption timeout for the root queue; overridden by minSharePreemptionTimeout
-   element in root queue.
-
- * <<A defaultFairSharePreemptionThreshold element>>, which sets the fair share
-   preemption threshold for the root queue; overridden by fairSharePreemptionThreshold
-   element in root queue.
-
- * <<A queueMaxAppsDefault element>>, which sets the default running app limit
-   for queues; overriden by maxRunningApps element in each queue.
-
- * <<A queueMaxAMShareDefault element>>, which sets the default AM resource
-   limit for queue; overriden by maxAMShare element in each queue.
-
- * <<A defaultQueueSchedulingPolicy element>>, which sets the default scheduling
-   policy for queues; overriden by the schedulingPolicy element in each queue
-   if specified. Defaults to "fair".
-
- * <<A queuePlacementPolicy element>>, which contains a list of rule elements
-   that tell the scheduler how to place incoming apps into queues. Rules
-   are applied in the order that they are listed. Rules may take arguments. All
-   rules accept the "create" argument, which indicates whether the rule can create
-   a new queue. "Create" defaults to true; if set to false and the rule would
-   place the app in a queue that is not configured in the allocations file, we
-   continue on to the next rule. The last rule must be one that can never issue a
-   continue.  Valid rules are:
-
-     * specified: the app is placed into the queue it requested.  If the app
-       requested no queue, i.e. it specified "default", we continue. If the app
-       requested a queue name starting or ending with period, i.e. names like
-       ".q1" or "q1." will be rejected.
-
-     * user: the app is placed into a queue with the name of the user who
-       submitted it. Periods in the username will be replace with "_dot_",
-       i.e. the queue name for user "first.last" is "first_dot_last".
-
-     * primaryGroup: the app is placed into a queue with the name of the
-       primary group of the user who submitted it. Periods in the group name
-       will be replaced with "_dot_", i.e. the queue name for group "one.two"
-       is "one_dot_two".
-
-     * secondaryGroupExistingQueue: the app is placed into a queue with a name
-       that matches a secondary group of the user who submitted it. The first
-       secondary group that matches a configured queue will be selected.
-       Periods in group names will be replaced with "_dot_", i.e. a user with
-       "one.two" as one of their secondary groups would be placed into the
-       "one_dot_two" queue, if such a queue exists.
-
-     * nestedUserQueue : the app is placed into a queue with the name of the user
-       under the queue suggested by the nested rule. This is similar to ‘user’
-       rule,the difference being in 'nestedUserQueue' rule,user queues can be created 
-       under any parent queue, while 'user' rule creates user queues only under root queue.
-       Note that nestedUserQueue rule would be applied only if the nested rule returns a 
-       parent queue.One can configure a parent queue either by setting 'type' attribute of queue
-       to 'parent' or by configuring at least one leaf under that queue which makes it a parent.
-       See example allocation for a sample use case. 
-
-     * default: the app is placed into the queue specified in the 'queue' attribute of the 
-       default rule. If 'queue' attribute is not specified, the app is placed into 'root.default' queue.
-
-     * reject: the app is rejected.
-
-  An example allocation file is given here:
-
----
-<?xml version="1.0"?>
-<allocations>
-  <queue name="sample_queue">
-    <minResources>10000 mb,0vcores</minResources>
-    <maxResources>90000 mb,0vcores</maxResources>
-    <maxRunningApps>50</maxRunningApps>
-    <maxAMShare>0.1</maxAMShare>
-    <weight>2.0</weight>
-    <schedulingPolicy>fair</schedulingPolicy>
-    <queue name="sample_sub_queue">
-      <aclSubmitApps>charlie</aclSubmitApps>
-      <minResources>5000 mb,0vcores</minResources>
-    </queue>
-  </queue>
-
-  <queueMaxAMShareDefault>0.5</queueMaxAMShareDefault>
-
-  <!-- Queue 'secondary_group_queue' is a parent queue and may have
-       user queues under it -->
-  <queue name="secondary_group_queue" type="parent">
-  <weight>3.0</weight>
-  </queue>
-  
-  <user name="sample_user">
-    <maxRunningApps>30</maxRunningApps>
-  </user>
-  <userMaxAppsDefault>5</userMaxAppsDefault>
-  
-  <queuePlacementPolicy>
-    <rule name="specified" />
-    <rule name="primaryGroup" create="false" />
-    <rule name="nestedUserQueue">
-        <rule name="secondaryGroupExistingQueue" create="false" />
-    </rule>
-    <rule name="default" queue="sample_queue"/>
-  </queuePlacementPolicy>
-</allocations>
----
-
-  Note that for backwards compatibility with the original FairScheduler, "queue" elements can instead be named as "pool" elements.
-
-
-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.
-
-  <<Note:>> The delimiter is a space character. To specify only ACL groups, begin the 
-  value with a space character. 
-  
-  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 "*". 
-
-  
-* {Administration}
-
-  The fair scheduler provides support for administration at runtime through a few mechanisms:
-
-Modifying configuration at runtime
-
-  It is possible to modify minimum shares, limits, weights, preemption timeouts
-  and queue scheduling policies at runtime by editing the allocation file. The
-  scheduler will reload this file 10-15 seconds after it sees that it was
-  modified.
-
-Monitoring through web UI
-
-  Current applications, queues, and fair shares can be examined through the
-  ResourceManager's web interface, at
-  http://<ResourceManager URL>/cluster/scheduler.
-
-  The following fields can be seen for each queue on the web interface:
-  
- * Used Resources - The sum of resources allocated to containers within the queue. 
-
- * Num Active Applications - The number of applications in the queue that have
-   received at least one container.
- 
- * Num Pending Applications - The number of applications in the queue that have
-   not yet received any containers.
-
- * Min Resources - The configured minimum resources that are guaranteed to the queue.
-  	
- * Max Resources - The configured maximum resources that are allowed to the queue.
- 
- * Instantaneous Fair Share - The queue's instantaneous fair share of resources.
-   These shares consider only actives queues (those with running applications),
-   and are used for scheduling decisions. Queues may be allocated resources
-   beyond their shares when other queues aren't using them. A queue whose
-   resource consumption lies at or below its instantaneous fair share will never
-   have its containers preempted.
-
- * Steady Fair Share - The queue's steady fair share of resources. These shares
-   consider all the queues irrespective of whether they are active (have
-   running applications) or not. These are computed less frequently and
-   change only when the configuration or capacity changes.They are meant to
-   provide visibility into resources the user can expect, and hence displayed
-   in the Web UI.
-
-Moving applications between queues
-
-  The Fair Scheduler supports moving a running application to a different queue.
-  This can be useful for moving an important application to a higher priority
-  queue, or for moving an unimportant application to a lower priority queue.
-  Apps can be moved by running "yarn application -movetoqueue appID -queue
-  targetQueueName".
-  
-  When an application is moved to a queue, its existing allocations become
-  counted with the new queue's allocations instead of the old for purposes
-  of determining fairness. An attempt to move an application to a queue will
-  fail if the addition of the app's resources to that queue would violate the
-  its maxRunningApps or maxResources constraints.
-

http://git-wip-us.apache.org/repos/asf/hadoop/blob/06aca7c6/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/apt/NodeManager.apt.vm
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/apt/NodeManager.apt.vm b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/apt/NodeManager.apt.vm
deleted file mode 100644
index 9ee942f..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/apt/NodeManager.apt.vm
+++ /dev/null
@@ -1,64 +0,0 @@
-~~ 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.
-
-  ---
-  NodeManager Overview.
-  ---
-  ---
-  ${maven.build.timestamp}
-
-NodeManager Overview.
-
-%{toc|section=1|fromDepth=0|toDepth=2}
-
-* Overview
-
-  The NodeManager is responsible for launching and managing containers on a node. Containers execute tasks as specified by the AppMaster.
-  
-* Health checker service
-
-  The NodeManager runs services to determine the health of the node it is executing on. The services perform checks on the disk as well as any user specified tests. If any health check fails, the NodeManager marks the node as unhealthy and communicates this to the ResourceManager, which then stops assigning containers to the node. Communication of the node status is done as part of the heartbeat between the NodeManager and the ResourceManager. The intervals at which the disk checker and health monitor(described below) run don't affect the heartbeat intervals. When the heartbeat takes place, the status of both checks is used to determine the health of the node.
-
-  ** Disk checker
-
-    The disk checker checks the state of the disks that the NodeManager is configured to use(local-dirs and log-dirs, configured using yarn.nodemanager.local-dirs and yarn.nodemanager.log-dirs respectively). The checks include permissions and free disk space. It also checks that the filesystem isn't in a read-only state. The checks are run at 2 minute intervals by default but can be configured to run as often as the user desires. If a disk fails the check, the NodeManager stops using that particular disk but still reports the node status as healthy. However if a number of disks fail the check(the number can be configured, as explained below), then the node is reported as unhealthy to the ResourceManager and new containers will not be assigned to the node. In addition, once a disk is marked as unhealthy, the NodeManager stops checking it to see if it has recovered(e.g. disk became full and was then cleaned up). The only way for the NodeManager to use that disk to restart the software
  on the node. The following configuration parameters can be used to modify the disk checks:
-
-*------------------+----------------+------------------+
-|| Configuration name || Allowed Values || Description |
-*------------------+----------------+------------------+
-| yarn.nodemanager.disk-health-checker.enable | true, false | Enable or disable the disk health checker service |
-*------------------+----------------+------------------+
-| yarn.nodemanager.disk-health-checker.interval-ms | Positive integer | The interval, in milliseconds, at which the disk checker should run; the default value is 2 minutes |
-*------------------+----------------+------------------+
-| yarn.nodemanager.disk-health-checker.min-healthy-disks | Float between 0-1 | The minimum fraction of disks that must pass the check for the NodeManager to mark the node as healthy; the default is 0.25 |
-*------------------+----------------+------------------+
-| yarn.nodemanager.disk-health-checker.max-disk-utilization-per-disk-percentage | Float between 0-100 | The maximum percentage of disk space that may be utilized before a disk is marked as unhealthy by the disk checker service. This check is run for every disk used by the NodeManager. The default value is 100 i.e. the entire disk can be used. |
-*------------------+----------------+------------------+
-| yarn.nodemanager.disk-health-checker.min-free-space-per-disk-mb | Integer | The minimum amount of free space that must be available on the disk for the disk checker service to mark the disk as healthy. This check is run for every disk used by the NodeManager. The default value is 0 i.e. the entire disk can be used. |
-*------------------+----------------+------------------+
-
- ** External health script
-
-    Users may specify their own health checker script that will be invoked by the health checker service. Users may specify a timeout as well as options to be passed to the script. If the script exits with a non-zero exit code, times out or results in an exception being thrown, the node is marked as unhealthy. Please note that if the script cannot be executed due to permissions or an incorrect path, etc, then it counts as a failure and the node will be reported as unhealthy. Please note that speifying a health check script is not mandatory. If no script is specified, only the disk checker status will be used to determine the health of the node. The following configuration parameters can be used to set the health script:
-
-*------------------+----------------+------------------+
-|| Configuration name || Allowed Values || Description |
-*------------------+----------------+------------------+
-| yarn.nodemanager.health-checker.interval-ms | Postive integer | The interval, in milliseconds, at which health checker service runs; the default value is 10 minutes. |
-*------------------+----------------+------------------+
-| yarn.nodemanager.health-checker.script.timeout-ms | Postive integer | The timeout for the health script that's executed; the default value is 20 minutes. |
-*------------------+----------------+------------------+
-| yarn.nodemanager.health-checker.script.path | String | Absolute path to the health check script to be run. |
-*------------------+----------------+------------------+
-| yarn.nodemanager.health-checker.script.opts | String | Arguments to be passed to the script when the script is executed. |
-*------------------+----------------+------------------+
-

http://git-wip-us.apache.org/repos/asf/hadoop/blob/06aca7c6/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/apt/NodeManagerCgroups.apt.vm
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/apt/NodeManagerCgroups.apt.vm b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/apt/NodeManagerCgroups.apt.vm
deleted file mode 100644
index f228e3b..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/apt/NodeManagerCgroups.apt.vm
+++ /dev/null
@@ -1,77 +0,0 @@
-~~ 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.
-
-  ---
-  Using CGroups with YARN
-  ---
-  ---
-  ${maven.build.timestamp}
-
-Using CGroups with YARN
-
-%{toc|section=1|fromDepth=0|toDepth=2}
-
- CGroups is a mechanism for aggregating/partitioning sets of tasks, and all their future children, into hierarchical groups with specialized behaviour. CGroups is a Linux kernel feature and was merged into kernel version 2.6.24. From a YARN perspective, this allows containers to be limited in their resource usage. A good example of this is CPU usage. Without CGroups, it becomes hard to limit container CPU usage. Currently, CGroups is only used for limiting CPU usage.
-
-* CGroups configuration
-
- The config variables related to using CGroups are the following:
-
- The following settings are related to setting up CGroups. All of these need to be set in yarn-site.xml.
-
-  [[1]] yarn.nodemanager.container-executor.class
-
-    This should be set to "org.apache.hadoop.yarn.server.nodemanager.LinuxContainerExecutor". CGroups is a Linux kernel feature and is exposed via the LinuxContainerExecutor.
-
-  [[2]] yarn.nodemanager.linux-container-executor.resources-handler.class
-
-    This should be set to "org.apache.hadoop.yarn.server.nodemanager.util.CgroupsLCEResourcesHandler".Using the LinuxContainerExecutor doesn't force you to use CGroups. If you wish to use CGroups, the resource-handler-class must be set to CGroupsLCEResourceHandler.
-
-  [[3]] yarn.nodemanager.linux-container-executor.cgroups.hierarchy
-
-    The cgroups hierarchy under which to place YARN proccesses(cannot contain commas). If yarn.nodemanager.linux-container-executor.cgroups.mount is false (that is, if cgroups have been pre-configured), then this cgroups hierarchy must already exist
-
-  [[4]] yarn.nodemanager.linux-container-executor.cgroups.mount
-
-    Whether the LCE should attempt to mount cgroups if not found - can be true or false
-
-  [[5]] yarn.nodemanager.linux-container-executor.cgroups.mount-path
-
-    Where the LCE should attempt to mount cgroups if not found. Common locations include /sys/fs/cgroup and /cgroup; the default location can vary depending on the Linux distribution in use. This path must exist before the NodeManager is launched. Only used when the LCE resources handler is set to the CgroupsLCEResourcesHandler, and yarn.nodemanager.linux-container-executor.cgroups.mount is true. A point to note here is that the container-executor binary will try to mount the path specified + "/" + the subsystem. In our case, since we are trying to limit CPU the binary tries to mount the path specified + "/cpu" and that's the path it expects to exist.
-
-  [[6]] yarn.nodemanager.linux-container-executor.group
-
-    The Unix group of the NodeManager. It should match the setting in "container-executor.cfg". This configuration is required for validating the secure access of the container-executor binary.
-
- The following settings are related to limiting resource usage of YARN containers
-
-  [[1]] yarn.nodemanager.resource.percentage-physical-cpu-limit
-
-    This setting lets you limit the cpu usage of all YARN containers. It sets a hard upper limit on the cumulative CPU usage of the containers. For example, if set to 60, the combined CPU usage of all YARN containers will not exceed 60%.
-
-  [[2]] yarn.nodemanager.linux-container-executor.cgroups.strict-resource-usage
-
-    CGroups allows cpu usage limits to be hard or soft. When this setting is true, containers cannot use more CPU usage than allocated even if spare CPU is available. This ensures that containers can only use CPU that they were allocated. When set to false, containers can use spare CPU if available. It should be noted that irrespective of whether set to true or false, at no time can the combined CPU usage of all containers exceed the value specified in "yarn.nodemanager.resource.percentage-physical-cpu-limit".
-
-* CGroups and security
-
- CGroups itself has no requirements related to security. However, the LinuxContainerExecutor does have some requirements. If running in non-secure mode, by default, the LCE runs all jobs as user "nobody". This user can be changed by setting "yarn.nodemanager.linux-container-executor.nonsecure-mode.local-user" to the desired user. However, it can also be configured to run jobs as the user submitting the job. In that case "yarn.nodemanager.linux-container-executor.nonsecure-mode.limit-users" should be set to false.
-
-*-----------+-----------+---------------------------+
-|| yarn.nodemanager.linux-container-executor.nonsecure-mode.local-user || yarn.nodemanager.linux-container-executor.nonsecure-mode.limit-users || User running jobs |
-*-----------+-----------+---------------------------+
-| (default) | (default) | nobody                    |
-*-----------+-----------+---------------------------+
-| yarn      | (default) | yarn                      |
-*-----------+-----------+---------------------------+
-| yarn      | false     | (User submitting the job) |
-*-----------+-----------+---------------------------+


[09/50] [abbrv] hadoop git commit: YARN-3256. TestClientToAMTokens#testClientTokenRace is not running against all Schedulers even when using ParameterizedSchedulerTestBase. Contributed by Anubhav Dhoot.

Posted by zh...@apache.org.
YARN-3256. TestClientToAMTokens#testClientTokenRace is not running against
all Schedulers even when using ParameterizedSchedulerTestBase. Contributed
by Anubhav Dhoot.


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

Branch: refs/heads/HDFS-7285
Commit: 982165cadc65f1a3819622a2e38deceae4b20097
Parents: 199a82d
Author: Devaraj K <de...@apache.org>
Authored: Thu Feb 26 15:45:41 2015 +0530
Committer: Zhe Zhang <zh...@cloudera.com>
Committed: Mon Mar 2 09:13:51 2015 -0800

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                                  | 4 ++++
 .../server/resourcemanager/security/TestClientToAMTokens.java    | 3 ---
 2 files changed, 4 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/982165ca/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index e5148eb..ac3cbb2 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -658,6 +658,10 @@ Release 2.7.0 - UNRELEASED
     YARN-3239. WebAppProxy does not support a final tracking url which has
     query fragments and params (Jian He via jlowe)
 
+    YARN-3256. TestClientToAMTokens#testClientTokenRace is not running against 
+    all Schedulers even when using ParameterizedSchedulerTestBase. 
+    (Anubhav Dhoot via devaraj)
+
 Release 2.6.0 - 2014-11-18
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/982165ca/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/security/TestClientToAMTokens.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/security/TestClientToAMTokens.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/security/TestClientToAMTokens.java
index 78bc728..499b4d1 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/security/TestClientToAMTokens.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/security/TestClientToAMTokens.java
@@ -33,7 +33,6 @@ import java.net.InetSocketAddress;
 import java.nio.ByteBuffer;
 import java.security.PrivilegedAction;
 import java.security.PrivilegedExceptionAction;
-import java.util.Arrays;
 import java.util.Timer;
 import java.util.TimerTask;
 
@@ -43,7 +42,6 @@ import org.junit.Assert;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
-import org.apache.hadoop.io.DataInputBuffer;
 import org.apache.hadoop.ipc.RPC;
 import org.apache.hadoop.ipc.RemoteException;
 import org.apache.hadoop.ipc.Server;
@@ -421,7 +419,6 @@ public class TestClientToAMTokens extends ParameterizedSchedulerTestBase {
   @Test(timeout=20000)
   public void testClientTokenRace() throws Exception {
 
-    final Configuration conf = new Configuration();
     conf.set(CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHENTICATION,
       "kerberos");
     UserGroupInformation.setConfiguration(conf);


[42/50] [abbrv] hadoop git commit: HDFS-5853. Add "hadoop.user.group.metrics.percentiles.intervals" to hdfs-default.xml (aajisaka)

Posted by zh...@apache.org.
HDFS-5853. Add "hadoop.user.group.metrics.percentiles.intervals" to hdfs-default.xml (aajisaka)


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

Branch: refs/heads/HDFS-7285
Commit: b1ddc7157d74161cee342405822bb93932a22cb4
Parents: 133d04c
Author: Akira Ajisaka <aa...@apache.org>
Authored: Sun Mar 1 01:16:36 2015 -0800
Committer: Zhe Zhang <zh...@cloudera.com>
Committed: Mon Mar 2 09:13:55 2015 -0800

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt              |  3 +++
 .../hadoop-hdfs/src/main/resources/hdfs-default.xml      | 11 +++++++++++
 2 files changed, 14 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/b1ddc715/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 16fe394..ce35ea2 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -691,6 +691,9 @@ Release 2.7.0 - UNRELEASED
     HDFS-7685. Document dfs.namenode.heartbeat.recheck-interval in
     hdfs-default.xml. (Kai Sasaki via aajisaka)
 
+    HDFS-5853. Add "hadoop.user.group.metrics.percentiles.intervals" to
+    hdfs-default.xml. (aajisaka)
+
   OPTIMIZATIONS
 
     HDFS-7454. Reduce memory footprint for AclEntries in NameNode.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b1ddc715/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml b/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
index 66fe86c..7eacfc5 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
@@ -1518,6 +1518,17 @@
 </property>
 
 <property>
+  <name>hadoop.user.group.metrics.percentiles.intervals</name>
+  <value></value>
+  <description>
+    A comma-separated list of the granularity in seconds for the metrics
+    which describe the 50/75/90/95/99th percentile latency for group resolution
+    in milliseconds.
+    By default, percentile latency metrics are disabled.
+  </description>
+</property>
+
+<property>
   <name>dfs.encrypt.data.transfer</name>
   <value>false</value>
   <description>


[50/50] [abbrv] hadoop git commit: HDFS-7789. DFSck should resolve the path to support cross-FS symlinks. (gera)

Posted by zh...@apache.org.
HDFS-7789. DFSck should resolve the path to support cross-FS symlinks. (gera)


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

Branch: refs/heads/HDFS-7285
Commit: 7deb79f95ef9a0efb983bde557a0ff0158b11d0d
Parents: 0e4a23f
Author: Gera Shegalov <ge...@apache.org>
Authored: Thu Feb 12 04:32:43 2015 -0800
Committer: Zhe Zhang <zh...@cloudera.com>
Committed: Mon Mar 2 09:13:56 2015 -0800

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |  3 ++
 .../org/apache/hadoop/hdfs/tools/DFSck.java     | 31 +++++++++++++-------
 .../hadoop/hdfs/server/namenode/TestFsck.java   | 14 ++++++---
 .../namenode/TestFsckWithMultipleNameNodes.java | 20 +++++++++++++
 4 files changed, 53 insertions(+), 15 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/7deb79f9/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 5ca16af..d5208da 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -697,6 +697,9 @@ Release 2.7.0 - UNRELEASED
     HDFS-7439. Add BlockOpResponseProto's message to the exception messages.
     (Takanobu Asanuma via szetszwo)
 
+    HDFS-7789. DFSck should resolve the path to support cross-FS symlinks.
+    (gera)
+
   OPTIMIZATIONS
 
     HDFS-7454. Reduce memory footprint for AclEntries in NameNode.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7deb79f9/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSck.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSck.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSck.java
index ec83a90..dc6d9d4 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSck.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSck.java
@@ -225,6 +225,14 @@ public class DFSck extends Configured implements Tool {
     return errCode;
   }
   
+
+  private Path getResolvedPath(String dir) throws IOException {
+    Configuration conf = getConf();
+    Path dirPath = new Path(dir);
+    FileSystem fs = dirPath.getFileSystem(conf);
+    return fs.resolvePath(dirPath);
+  }
+
   /**
    * Derive the namenode http address from the current file system,
    * either default or as set by "-fs" in the generic options.
@@ -236,19 +244,12 @@ public class DFSck extends Configured implements Tool {
     Configuration conf = getConf();
 
     //get the filesystem object to verify it is an HDFS system
-    final FileSystem fs;
-    try {
-      fs = target.getFileSystem(conf);
-    } catch (IOException ioe) {
-      System.err.println("FileSystem is inaccessible due to:\n"
-          + StringUtils.stringifyException(ioe));
-      return null;
-    }
+    final FileSystem fs = target.getFileSystem(conf);
     if (!(fs instanceof DistributedFileSystem)) {
       System.err.println("FileSystem is " + fs.getUri());
       return null;
     }
-    
+
     return DFSUtil.getInfoServer(HAUtil.getAddressOfActive(fs), conf,
         DFSUtil.getHttpClientScheme(conf));
   }
@@ -303,8 +304,16 @@ public class DFSck extends Configured implements Tool {
       dir = "/";
     }
 
-    final Path dirpath = new Path(dir);
-    final URI namenodeAddress = getCurrentNamenodeAddress(dirpath);
+    Path dirpath = null;
+    URI namenodeAddress = null;
+    try {
+      dirpath = getResolvedPath(dir);
+      namenodeAddress = getCurrentNamenodeAddress(dirpath);
+    } catch (IOException ioe) {
+      System.err.println("FileSystem is inaccessible due to:\n"
+          + StringUtils.stringifyException(ioe));
+    }
+
     if (namenodeAddress == null) {
       //Error message already output in {@link #getCurrentNamenodeAddress()}
       System.err.println("DFSck exiting.");

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7deb79f9/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFsck.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFsck.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFsck.java
index 33de692..0b23b84 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFsck.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFsck.java
@@ -211,10 +211,16 @@ public class TestFsck {
     try {
       // Audit log should contain one getfileinfo and one fsck
       reader = new BufferedReader(new FileReader(auditLogFile));
-      String line = reader.readLine();
-      assertNotNull(line);
-      assertTrue("Expected getfileinfo event not found in audit log",
-          getfileinfoPattern.matcher(line).matches());
+      String line;
+
+      // one extra getfileinfo stems from resolving the path
+      //
+      for (int i = 0; i < 2; i++) {
+        line = reader.readLine();
+        assertNotNull(line);
+        assertTrue("Expected getfileinfo event not found in audit log",
+            getfileinfoPattern.matcher(line).matches());
+      }
       line = reader.readLine();
       assertNotNull(line);
       assertTrue("Expected fsck event not found in audit log", fsckPattern

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7deb79f9/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFsckWithMultipleNameNodes.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFsckWithMultipleNameNodes.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFsckWithMultipleNameNodes.java
index f4cb624..124b301 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFsckWithMultipleNameNodes.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFsckWithMultipleNameNodes.java
@@ -18,6 +18,7 @@
 package org.apache.hadoop.hdfs.server.namenode;
 
 import java.io.IOException;
+import java.net.URI;
 import java.util.Random;
 import java.util.concurrent.TimeoutException;
 
@@ -26,6 +27,8 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.viewfs.ConfigUtil;
+import org.apache.hadoop.fs.viewfs.ViewFileSystem;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
@@ -119,6 +122,23 @@ public class TestFsckWithMultipleNameNodes {
         LOG.info("result=" + result);
         Assert.assertTrue(result.contains("Status: HEALTHY"));
       }
+
+      // Test viewfs
+      //
+      LOG.info("RUN_TEST 3");
+      final String[] vurls = new String[nNameNodes];
+      for (int i = 0; i < vurls.length; i++) {
+        String link = "/mount/nn_" + i + FILE_NAME;
+        ConfigUtil.addLink(conf, link, new URI(urls[i]));
+        vurls[i] = "viewfs:" + link;
+      }
+
+      for(int i = 0; i < vurls.length; i++) {
+        LOG.info("vurls[" + i + "]=" + vurls[i]);
+        final String result = TestFsck.runFsck(conf, 0, false, vurls[i]);
+        LOG.info("result=" + result);
+        Assert.assertTrue(result.contains("Status: HEALTHY"));
+      }
     } finally {
       cluster.shutdown();
     }


[06/50] [abbrv] hadoop git commit: HADOOP-11629. WASB filesystem should not start BandwidthGaugeUpdater if fs.azure.skip.metrics set to true. Contributed by Shanyu Zhao.

Posted by zh...@apache.org.
HADOOP-11629. WASB filesystem should not start BandwidthGaugeUpdater if fs.azure.skip.metrics set to true. Contributed by Shanyu Zhao.


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

Branch: refs/heads/HDFS-7285
Commit: 8d184d1288df580f574b9c60bb7c19afb938a5a7
Parents: 4b09124
Author: cnauroth <cn...@apache.org>
Authored: Wed Feb 25 09:08:55 2015 -0800
Committer: Zhe Zhang <zh...@cloudera.com>
Committed: Mon Mar 2 09:13:50 2015 -0800

----------------------------------------------------------------------
 hadoop-common-project/hadoop-common/CHANGES.txt |  3 ++
 .../fs/azure/AzureNativeFileSystemStore.java    | 29 ++++++++++++++------
 2 files changed, 23 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/8d184d12/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 988eed0..0d452f7 100644
--- a/hadoop-common-project/hadoop-common/CHANGES.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES.txt
@@ -1002,6 +1002,9 @@ Release 2.7.0 - UNRELEASED
     HADOOP-11480. Typo in hadoop-aws/index.md uses wrong scheme for
     test.fs.s3.name. (Ted Yu via aajisaka)
 
+    HADOOP-11629. WASB filesystem should not start BandwidthGaugeUpdater if
+    fs.azure.skip.metrics set to true. (Shanyu Zhao via cnauroth)
+
 Release 2.6.1 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8d184d12/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/AzureNativeFileSystemStore.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/AzureNativeFileSystemStore.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/AzureNativeFileSystemStore.java
index 2412698..6bed8bb 100644
--- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/AzureNativeFileSystemStore.java
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/AzureNativeFileSystemStore.java
@@ -387,9 +387,8 @@ public class AzureNativeFileSystemStore implements NativeFileSystemStore {
     if (null == instrumentation) {
       throw new IllegalArgumentException("Null instrumentation");
     }
-
     this.instrumentation = instrumentation;
-    this.bandwidthGaugeUpdater = new BandwidthGaugeUpdater(instrumentation);
+
     if (null == this.storageInteractionLayer) {
       this.storageInteractionLayer = new StorageInterfaceImpl();
     }
@@ -405,7 +404,13 @@ public class AzureNativeFileSystemStore implements NativeFileSystemStore {
     //
     if (null == conf) {
       throw new IllegalArgumentException(
-          "Cannot initialize WASB file system, URI is null");
+          "Cannot initialize WASB file system, conf is null");
+    }
+
+    if(!conf.getBoolean(
+        NativeAzureFileSystem.SKIP_AZURE_METRICS_PROPERTY_NAME, false)) {
+      //If not skip azure metrics, create bandwidthGaugeUpdater
+      this.bandwidthGaugeUpdater = new BandwidthGaugeUpdater(instrumentation);
     }
 
     // Incoming parameters validated. Capture the URI and the job configuration
@@ -1782,11 +1787,14 @@ public class AzureNativeFileSystemStore implements NativeFileSystemStore {
           selfThrottlingWriteFactor);
     }
 
-    ResponseReceivedMetricUpdater.hook(
-        operationContext,
-        instrumentation,
-        bandwidthGaugeUpdater);
-    
+    if(bandwidthGaugeUpdater != null) {
+      //bandwidthGaugeUpdater is null when we config to skip azure metrics
+      ResponseReceivedMetricUpdater.hook(
+         operationContext,
+         instrumentation,
+         bandwidthGaugeUpdater);
+    }
+
     // Bind operation context to receive send request callbacks on this operation.
     // If reads concurrent to OOB writes are allowed, the interception will reset
     // the conditional header on all Azure blob storage read requests.
@@ -2561,7 +2569,10 @@ public class AzureNativeFileSystemStore implements NativeFileSystemStore {
 
   @Override
   public void close() {
-    bandwidthGaugeUpdater.close();
+    if(bandwidthGaugeUpdater != null) {
+      bandwidthGaugeUpdater.close();
+      bandwidthGaugeUpdater = null;
+    }
   }
   
   // Finalizer to ensure complete shutdown


[16/50] [abbrv] hadoop git commit: HADOOP-11637. bash location hard-coded in shell scripts (aw)

Posted by zh...@apache.org.
HADOOP-11637. bash location hard-coded in shell scripts (aw)


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

Branch: refs/heads/HDFS-7285
Commit: db0b6e60d0e20ff6207359ed454d98403f36dada
Parents: 41d9952
Author: Allen Wittenauer <aw...@apache.org>
Authored: Thu Feb 26 09:29:16 2015 -0800
Committer: Zhe Zhang <zh...@cloudera.com>
Committed: Mon Mar 2 09:13:52 2015 -0800

----------------------------------------------------------------------
 hadoop-common-project/hadoop-common/CHANGES.txt                 | 2 ++
 .../hadoop-common/src/main/bin/hadoop-functions.sh              | 2 +-
 .../hadoop-common/src/site/markdown/RackAwareness.md            | 5 +++--
 hadoop-common-project/hadoop-kms/src/main/conf/kms-env.sh       | 2 +-
 hadoop-common-project/hadoop-kms/src/main/libexec/kms-config.sh | 2 +-
 hadoop-common-project/hadoop-kms/src/main/sbin/kms.sh           | 2 +-
 .../hadoop-hdfs-httpfs/src/main/conf/httpfs-env.sh              | 2 +-
 .../hadoop-hdfs-httpfs/src/main/libexec/httpfs-config.sh        | 2 +-
 hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/sbin/httpfs.sh  | 2 +-
 hadoop-tools/hadoop-sls/src/main/bin/rumen2sls.sh               | 2 +-
 hadoop-tools/hadoop-sls/src/main/bin/slsrun.sh                  | 2 +-
 11 files changed, 14 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/db0b6e60/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 39062a8..ca27463 100644
--- a/hadoop-common-project/hadoop-common/CHANGES.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES.txt
@@ -404,6 +404,8 @@ Trunk (Unreleased)
 
     HADOOP-11625. Minor fixes to command manual & SLA doc (aw)
 
+    HADOOP-11637. bash location hard-coded in shell scripts (aw)
+
   OPTIMIZATIONS
 
     HADOOP-7761. Improve the performance of raw comparisons. (todd)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/db0b6e60/hadoop-common-project/hadoop-common/src/main/bin/hadoop-functions.sh
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/bin/hadoop-functions.sh b/hadoop-common-project/hadoop-common/src/main/bin/hadoop-functions.sh
index cec6b2c..bccbe25 100644
--- a/hadoop-common-project/hadoop-common/src/main/bin/hadoop-functions.sh
+++ b/hadoop-common-project/hadoop-common/src/main/bin/hadoop-functions.sh
@@ -1,4 +1,4 @@
-#!/bin/bash
+#!/usr/bin/env bash
 # Licensed to the Apache Software Foundation (ASF) under one or more
 # contributor license agreements.  See the NOTICE file distributed with
 # this work for additional information regarding copyright ownership.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/db0b6e60/hadoop-common-project/hadoop-common/src/site/markdown/RackAwareness.md
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/site/markdown/RackAwareness.md b/hadoop-common-project/hadoop-common/src/site/markdown/RackAwareness.md
index c5ab19a..09f5610 100644
--- a/hadoop-common-project/hadoop-common/src/site/markdown/RackAwareness.md
+++ b/hadoop-common-project/hadoop-common/src/site/markdown/RackAwareness.md
@@ -105,7 +105,7 @@ bash Example
 ------------
 
 ```bash
-#!/bin/bash
+#!/usr/bin/env bash
 # Here's a bash example to show just how simple these scripts can be
 # Assuming we have flat network with everything on a single switch, we can fake a rack topology.
 # This could occur in a lab environment where we have limited nodes,like 2-8 physical machines on a unmanaged switch.
@@ -133,4 +133,5 @@ bash Example
 #    fails to split on four dots, it will still print '/rack-' last field value
 
 echo $@ | xargs -n 1 | awk -F '.' '{print "/rack-"$NF}'
-```
\ No newline at end of file
+```
+

http://git-wip-us.apache.org/repos/asf/hadoop/blob/db0b6e60/hadoop-common-project/hadoop-kms/src/main/conf/kms-env.sh
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-kms/src/main/conf/kms-env.sh b/hadoop-common-project/hadoop-kms/src/main/conf/kms-env.sh
index de9554f..41449ef 100644
--- a/hadoop-common-project/hadoop-kms/src/main/conf/kms-env.sh
+++ b/hadoop-common-project/hadoop-kms/src/main/conf/kms-env.sh
@@ -1,4 +1,4 @@
-#!/bin/bash
+#!/usr/bin/env bash
 #
 # Licensed under the Apache License, Version 2.0 (the "License");
 # you may not use this file except in compliance with the License.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/db0b6e60/hadoop-common-project/hadoop-kms/src/main/libexec/kms-config.sh
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-kms/src/main/libexec/kms-config.sh b/hadoop-common-project/hadoop-kms/src/main/libexec/kms-config.sh
index c1aa136..df48889 100644
--- a/hadoop-common-project/hadoop-kms/src/main/libexec/kms-config.sh
+++ b/hadoop-common-project/hadoop-kms/src/main/libexec/kms-config.sh
@@ -1,4 +1,4 @@
-#!/bin/bash
+#!/usr/bin/env bash
 #
 # Licensed under the Apache License, Version 2.0 (the "License");
 # you may not use this file except in compliance with the License.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/db0b6e60/hadoop-common-project/hadoop-kms/src/main/sbin/kms.sh
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-kms/src/main/sbin/kms.sh b/hadoop-common-project/hadoop-kms/src/main/sbin/kms.sh
index e4d4f93..9228d2d 100644
--- a/hadoop-common-project/hadoop-kms/src/main/sbin/kms.sh
+++ b/hadoop-common-project/hadoop-kms/src/main/sbin/kms.sh
@@ -1,4 +1,4 @@
-#!/bin/bash
+#!/usr/bin/env bash
 #
 # Licensed under the Apache License, Version 2.0 (the "License");
 # you may not use this file except in compliance with the License.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/db0b6e60/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/conf/httpfs-env.sh
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/conf/httpfs-env.sh b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/conf/httpfs-env.sh
index 0e8cc40..a4edef6 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/conf/httpfs-env.sh
+++ b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/conf/httpfs-env.sh
@@ -1,4 +1,4 @@
-#!/bin/bash
+#!/usr/bin/env bash
 #
 # Licensed under the Apache License, Version 2.0 (the "License");
 # you may not use this file except in compliance with the License.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/db0b6e60/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/libexec/httpfs-config.sh
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/libexec/httpfs-config.sh b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/libexec/httpfs-config.sh
index e520e08..767bd6e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/libexec/httpfs-config.sh
+++ b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/libexec/httpfs-config.sh
@@ -1,4 +1,4 @@
-#!/bin/bash
+#!/usr/bin/env bash
 #
 # Licensed under the Apache License, Version 2.0 (the "License");
 # you may not use this file except in compliance with the License.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/db0b6e60/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/sbin/httpfs.sh
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/sbin/httpfs.sh b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/sbin/httpfs.sh
index 72dca93..f51a5e6 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/sbin/httpfs.sh
+++ b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/sbin/httpfs.sh
@@ -1,4 +1,4 @@
-#!/bin/bash
+#!/usr/bin/env bash
 #
 # Licensed under the Apache License, Version 2.0 (the "License");
 # you may not use this file except in compliance with the License.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/db0b6e60/hadoop-tools/hadoop-sls/src/main/bin/rumen2sls.sh
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-sls/src/main/bin/rumen2sls.sh b/hadoop-tools/hadoop-sls/src/main/bin/rumen2sls.sh
index c8e38ac..f4fdabd 100644
--- a/hadoop-tools/hadoop-sls/src/main/bin/rumen2sls.sh
+++ b/hadoop-tools/hadoop-sls/src/main/bin/rumen2sls.sh
@@ -1,4 +1,4 @@
-#!/bin/bash
+#!/usr/bin/env bash
 #
 #  Licensed under the Apache License, Version 2.0 (the "License");
 #  you may not use this file except in compliance with the License.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/db0b6e60/hadoop-tools/hadoop-sls/src/main/bin/slsrun.sh
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-sls/src/main/bin/slsrun.sh b/hadoop-tools/hadoop-sls/src/main/bin/slsrun.sh
index 463ae23..d1409bf 100644
--- a/hadoop-tools/hadoop-sls/src/main/bin/slsrun.sh
+++ b/hadoop-tools/hadoop-sls/src/main/bin/slsrun.sh
@@ -1,4 +1,4 @@
-#!/bin/bash
+#!/usr/bin/env bash
 #
 #  Licensed under the Apache License, Version 2.0 (the "License");
 #  you may not use this file except in compliance with the License.


[27/50] [abbrv] hadoop git commit: recommit "HDFS-7769. TestHDFSCLI should not create files in hdfs project root dir." (cherry picked from commit 7c6b6547eeed110e1a842e503bfd33afe04fa814)

Posted by zh...@apache.org.
recommit "HDFS-7769. TestHDFSCLI should not create files in hdfs project root dir."
(cherry picked from commit 7c6b6547eeed110e1a842e503bfd33afe04fa814)


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

Branch: refs/heads/HDFS-7285
Commit: 97c2c59c611336401c2d11d72fc4434fcf607823
Parents: 70c1fa0
Author: Tsz-Wo Nicholas Sze <sz...@hortonworks.com>
Authored: Tue Feb 10 17:48:57 2015 -0800
Committer: Zhe Zhang <zh...@cloudera.com>
Committed: Mon Mar 2 09:13:54 2015 -0800

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt                      | 3 +++
 .../hadoop-hdfs/src/test/resources/testHDFSConf.xml              | 4 ++--
 2 files changed, 5 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/97c2c59c/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 b4b0087..2a8da43 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -981,6 +981,9 @@ Release 2.7.0 - UNRELEASED
     HDFS-7714. Simultaneous restart of HA NameNodes and DataNode can cause
     DataNode to register successfully with only one NameNode.(vinayakumarb)
 
+    HDFS-7769. TestHDFSCLI should not create files in hdfs project root dir.
+    (szetszwo)
+
     HDFS-7753. Fix Multithreaded correctness Warnings in BackupImage.
     (Rakesh R and shv)
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/97c2c59c/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testHDFSConf.xml
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testHDFSConf.xml b/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testHDFSConf.xml
index e59b05a..2d3de1f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testHDFSConf.xml
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testHDFSConf.xml
@@ -16483,8 +16483,8 @@
         <command>-fs NAMENODE -mkdir -p /user/USERNAME/dir1</command>
         <command>-fs NAMENODE -copyFromLocal CLITEST_DATA/data15bytes /user/USERNAME/dir1</command>
         <command>-fs NAMENODE -copyFromLocal CLITEST_DATA/data30bytes /user/USERNAME/dir1</command>
-        <command>-fs NAMENODE -getmerge /user/USERNAME/dir1 data</command>
-        <command>-cat data</command>
+        <command>-fs NAMENODE -getmerge /user/USERNAME/dir1 CLITEST_DATA/file</command>
+        <command>-cat CLITEST_DATA/file</command>
       </test-commands>
       <cleanup-commands>
         <command>-fs NAMENODE -rm -r /user/USERNAME</command>


[12/50] [abbrv] hadoop git commit: HDFS-7460. Rewrite httpfs to use new shell framework (John Smith via aw)

Posted by zh...@apache.org.
HDFS-7460. Rewrite httpfs to use new shell framework (John Smith via aw)


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

Branch: refs/heads/HDFS-7285
Commit: 21e9e91c8ab1160a9a756b5f155c62492661bc78
Parents: 746bc37
Author: Allen Wittenauer <aw...@apache.org>
Authored: Wed Feb 25 18:57:41 2015 -0800
Committer: Zhe Zhang <zh...@cloudera.com>
Committed: Mon Mar 2 09:13:51 2015 -0800

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs-httpfs/pom.xml  |   2 +-
 .../src/main/conf/httpfs-env.sh                 |  51 +++--
 .../src/main/libexec/httpfs-config.sh           | 208 ++++---------------
 .../hadoop-hdfs-httpfs/src/main/sbin/httpfs.sh  | 116 +++++++----
 .../src/main/tomcat/ssl-server.xml              | 135 ------------
 .../src/main/tomcat/ssl-server.xml.conf         | 135 ++++++++++++
 .../src/site/apt/ServerSetup.apt.vm             | 159 --------------
 .../src/site/apt/UsingHttpTools.apt.vm          |  87 --------
 .../src/site/apt/index.apt.vm                   |  83 --------
 .../src/site/markdown/ServerSetup.md.vm         | 121 +++++++++++
 .../src/site/markdown/UsingHttpTools.md         |  62 ++++++
 .../src/site/markdown/index.md                  |  52 +++++
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |   2 +
 13 files changed, 533 insertions(+), 680 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/21e9e91c/hadoop-hdfs-project/hadoop-hdfs-httpfs/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-httpfs/pom.xml b/hadoop-hdfs-project/hadoop-hdfs-httpfs/pom.xml
index 4c42ef9..ddc6033 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-httpfs/pom.xml
+++ b/hadoop-hdfs-project/hadoop-hdfs-httpfs/pom.xml
@@ -524,7 +524,7 @@
                     <copy file="${basedir}/src/main/tomcat/server.xml"
                           toDir="${httpfs.tomcat.dist.dir}/conf"/>
                     <delete file="${httpfs.tomcat.dist.dir}/conf/ssl-server.xml"/>
-                    <copy file="${basedir}/src/main/tomcat/ssl-server.xml"
+                    <copy file="${basedir}/src/main/tomcat/ssl-server.xml.conf"
                           toDir="${httpfs.tomcat.dist.dir}/conf"/>
                     <delete file="${httpfs.tomcat.dist.dir}/conf/logging.properties"/>
                     <copy file="${basedir}/src/main/tomcat/logging.properties"

http://git-wip-us.apache.org/repos/asf/hadoop/blob/21e9e91c/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/conf/httpfs-env.sh
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/conf/httpfs-env.sh b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/conf/httpfs-env.sh
index a2701d4..0e8cc40 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/conf/httpfs-env.sh
+++ b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/conf/httpfs-env.sh
@@ -14,40 +14,59 @@
 #
 
 # Set httpfs specific environment variables here.
-
-# Settings for the Embedded Tomcat that runs HttpFS
-# Java System properties for HttpFS should be specified in this variable
 #
-# export CATALINA_OPTS=
-
-# HttpFS logs directory
+# hadoop-env.sh is read prior to this file.
 #
-# export HTTPFS_LOG=${HTTPFS_HOME}/logs
 
-# HttpFS temporary directory
+# HTTPFS temporary directory
 #
-# export HTTPFS_TEMP=${HTTPFS_HOME}/temp
+# export HTTPFS_TEMP=${HADOOP_PREFIX}/temp
 
-# The HTTP port used by HttpFS
+# The HTTP port used by HTTPFS
 #
 # export HTTPFS_HTTP_PORT=14000
 
-# The Admin port used by HttpFS
+# The Admin port used by HTTPFS
 #
-# export HTTPFS_ADMIN_PORT=`expr ${HTTPFS_HTTP_PORT} + 1`
+# export HTTPFS_ADMIN_PORT=$((HTTPFS_HTTP_PORT + 1))
 
-# The hostname HttpFS server runs on
+# The maximum number of Tomcat handler threads
 #
-# export HTTPFS_HTTP_HOSTNAME=`hostname -f`
+# export HTTPFS_MAX_THREADS=1000
 
-# Indicates if HttpFS is using SSL
+# The hostname HttpFS server runs on
 #
-# export HTTPFS_SSL_ENABLED=false
+# export HTTPFS_HTTP_HOSTNAME=$(hostname -f)
 
 # The location of the SSL keystore if using SSL
 #
 # export HTTPFS_SSL_KEYSTORE_FILE=${HOME}/.keystore
 
+#
 # The password of the SSL keystore if using SSL
 #
 # export HTTPFS_SSL_KEYSTORE_PASS=password
+
+##
+## Tomcat specific settings
+##
+#
+# Location of tomcat
+#
+# export HTTPFS_CATALINA_HOME=${HADOOP_PREFIX}/share/hadoop/httpfs/tomcat
+
+# Java System properties for HTTPFS should be specified in this variable.
+# The java.library.path and hadoop.home.dir properties are automatically
+# configured.  In order to supplement java.library.path,
+# one should add to the JAVA_LIBRARY_PATH env var.
+#
+# export CATALINA_OPTS=
+
+# PID file
+#
+# export CATALINA_PID=${HADOOP_PID_DIR}/hadoop-${HADOOP_IDENT_STRING}-httpfs.pid
+
+# Output file
+#
+# export CATALINA_OUT=${HTTPFS_LOG}/hadoop-${HADOOP_IDENT_STRING}-httpfs-${HOSTNAME}.out
+

http://git-wip-us.apache.org/repos/asf/hadoop/blob/21e9e91c/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/libexec/httpfs-config.sh
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/libexec/httpfs-config.sh b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/libexec/httpfs-config.sh
index d4db28f..e520e08 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/libexec/httpfs-config.sh
+++ b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/libexec/httpfs-config.sh
@@ -1,4 +1,4 @@
-#!/usr/bin/env bash
+#!/bin/bash
 #
 # Licensed under the Apache License, Version 2.0 (the "License");
 # you may not use this file except in compliance with the License.
@@ -13,183 +13,63 @@
 #  limitations under the License.
 #
 
-# resolve links - $0 may be a softlink
-PRG="${0}"
-
-while [ -h "${PRG}" ]; do
-  ls=`ls -ld "${PRG}"`
-  link=`expr "$ls" : '.*-> \(.*\)$'`
-  if expr "$link" : '/.*' > /dev/null; then
-    PRG="$link"
-  else
-    PRG=`dirname "${PRG}"`/"$link"
+function hadoop_subproject_init
+{
+  local this
+  local binparent
+  local varlist
+
+  if [[ -z "${HADOOP_HTTPFS_ENV_PROCESSED}" ]]; then
+    if [[ -e "${HADOOP_CONF_DIR}/httpfs-env.sh" ]]; then
+      . "${HADOOP_CONF_DIR}/httpfs-env.sh"
+      export HADOOP_HTTPFS_ENV_PROCESSED=true
+    fi
   fi
-done
 
-BASEDIR=`dirname ${PRG}`
-BASEDIR=`cd ${BASEDIR}/..;pwd`
+  export HADOOP_CATALINA_PREFIX=httpfs
 
+  export HADOOP_CATALINA_TEMP="${HTTPFS_TEMP:-${HADOOP_PREFIX}/temp}"
 
-function print() {
-  if [ "${HTTPFS_SILENT}" != "true" ]; then
-    echo "$@"
-  fi
-}
+  hadoop_deprecate_envvar HTTPFS_CONFIG HADOOP_CONF_DIR
 
-# if HTTPFS_HOME is already set warn it will be ignored
-#
-if [ "${HTTPFS_HOME}" != "" ]; then
-  echo "WARNING: current setting of HTTPFS_HOME ignored"
-fi
+  hadoop_deprecate_envvar HTTPFS_LOG HADOOP_LOG_DIR
 
-print
+  export HADOOP_CATALINA_CONFIG="${HADOOP_CONF_DIR}"
+  export HADOOP_CATALINA_LOG="${HADOOP_LOG_DIR}"
 
-# setting HTTPFS_HOME to the installation dir, it cannot be changed
-#
-export HTTPFS_HOME=${BASEDIR}
-httpfs_home=${HTTPFS_HOME}
-print "Setting HTTPFS_HOME:          ${HTTPFS_HOME}"
+  export HTTPFS_HTTP_HOSTNAME=${HTTPFS_HTTP_HOSTNAME:-$(hostname -f)}
 
-# if the installation has a env file, source it
-# this is for native packages installations
-#
-if [ -e "${HTTPFS_HOME}/bin/httpfs-env.sh" ]; then
-  print "Sourcing:                    ${HTTPFS_HOME}/bin/httpfs-env.sh"
-  source ${HTTPFS_HOME}/bin/httpfs-env.sh
-  grep "^ *export " ${HTTPFS_HOME}/bin/httpfs-env.sh | sed 's/ *export/  setting/'
-fi
+  export HADOOP_CATALINA_HTTP_PORT="${HTTPFS_HTTP_PORT:-14000}"
+  export HADOOP_CATALINA_ADMIN_PORT="${HTTPFS_ADMIN_PORT:-$((HADOOP_CATALINA_HTTP_PORT+1))}"
+  export HADOOP_CATALINA_MAX_THREADS="${HTTPFS_MAX_THREADS:-150}"
 
-# verify that the sourced env file didn't change HTTPFS_HOME
-# if so, warn and revert
-#
-if [ "${HTTPFS_HOME}" != "${httpfs_home}" ]; then
-  print "WARN: HTTPFS_HOME resetting to ''${HTTPFS_HOME}'' ignored"
-  export HTTPFS_HOME=${httpfs_home}
-  print "  using HTTPFS_HOME:        ${HTTPFS_HOME}"
-fi
+  export HTTPFS_SSL_ENABLED=${HTTPFS_SSL_ENABLED:-false}
 
-if [ "${HTTPFS_CONFIG}" = "" ]; then
-  export HTTPFS_CONFIG=${HTTPFS_HOME}/etc/hadoop
-  print "Setting HTTPFS_CONFIG:        ${HTTPFS_CONFIG}"
-else
-  print "Using   HTTPFS_CONFIG:        ${HTTPFS_CONFIG}"
-fi
-httpfs_config=${HTTPFS_CONFIG}
+  export HADOOP_CATALINA_SSL_KEYSTORE_FILE="${HTTPFS_SSL_KEYSTORE_FILE:-${HOME}/.keystore}"
 
-# if the configuration dir has a env file, source it
-#
-if [ -e "${HTTPFS_CONFIG}/httpfs-env.sh" ]; then
-  print "Sourcing:                    ${HTTPFS_CONFIG}/httpfs-env.sh"
-  source ${HTTPFS_CONFIG}/httpfs-env.sh
-  grep "^ *export " ${HTTPFS_CONFIG}/httpfs-env.sh | sed 's/ *export/  setting/'
-fi
+  export CATALINA_BASE="${CATALINA_BASE:-${HADOOP_PREFIX}/share/hadoop/httpfs/tomcat}"
+  export HADOOP_CATALINA_HOME="${HTTPFS_CATALINA_HOME:-${CATALINA_BASE}}"
 
-# verify that the sourced env file didn't change HTTPFS_HOME
-# if so, warn and revert
-#
-if [ "${HTTPFS_HOME}" != "${httpfs_home}" ]; then
-  echo "WARN: HTTPFS_HOME resetting to ''${HTTPFS_HOME}'' ignored"
-  export HTTPFS_HOME=${httpfs_home}
-fi
-
-# verify that the sourced env file didn't change HTTPFS_CONFIG
-# if so, warn and revert
-#
-if [ "${HTTPFS_CONFIG}" != "${httpfs_config}" ]; then
-  echo "WARN: HTTPFS_CONFIG resetting to ''${HTTPFS_CONFIG}'' ignored"
-  export HTTPFS_CONFIG=${httpfs_config}
-fi
-
-if [ "${HTTPFS_LOG}" = "" ]; then
-  export HTTPFS_LOG=${HTTPFS_HOME}/logs
-  print "Setting HTTPFS_LOG:           ${HTTPFS_LOG}"
-else
-  print "Using   HTTPFS_LOG:           ${HTTPFS_LOG}"
-fi
+  export CATALINA_OUT="${CATALINA_OUT:-${HADOOP_LOG_DIR}/hadoop-${HADOOP_IDENT_STRING}-httpfs-${HOSTNAME}.out}"
 
-if [ ! -f ${HTTPFS_LOG} ]; then
-  mkdir -p ${HTTPFS_LOG}
-fi
-
-if [ "${HTTPFS_TEMP}" = "" ]; then
-  export HTTPFS_TEMP=${HTTPFS_HOME}/temp
-  print "Setting HTTPFS_TEMP:           ${HTTPFS_TEMP}"
-else
-  print "Using   HTTPFS_TEMP:           ${HTTPFS_TEMP}"
-fi
+  export CATALINA_PID="${CATALINA_PID:-${HADOOP_PID_DIR}/hadoop-${HADOOP_IDENT_STRING}-httpfs.pid}"
 
-if [ ! -f ${HTTPFS_TEMP} ]; then
-  mkdir -p ${HTTPFS_TEMP}
-fi
-
-if [ "${HTTPFS_HTTP_PORT}" = "" ]; then
-  export HTTPFS_HTTP_PORT=14000
-  print "Setting HTTPFS_HTTP_PORT:     ${HTTPFS_HTTP_PORT}"
-else
-  print "Using   HTTPFS_HTTP_PORT:     ${HTTPFS_HTTP_PORT}"
-fi
-
-if [ "${HTTPFS_ADMIN_PORT}" = "" ]; then
-  export HTTPFS_ADMIN_PORT=`expr $HTTPFS_HTTP_PORT +  1`
-  print "Setting HTTPFS_ADMIN_PORT:     ${HTTPFS_ADMIN_PORT}"
-else
-  print "Using   HTTPFS_ADMIN_PORT:     ${HTTPFS_ADMIN_PORT}"
-fi
-
-if [ "${HTTPFS_HTTP_HOSTNAME}" = "" ]; then
-  export HTTPFS_HTTP_HOSTNAME=`hostname -f`
-  print "Setting HTTPFS_HTTP_HOSTNAME: ${HTTPFS_HTTP_HOSTNAME}"
-else
-  print "Using   HTTPFS_HTTP_HOSTNAME: ${HTTPFS_HTTP_HOSTNAME}"
-fi
-
-if [ "${HTTPFS_SSL_ENABLED}" = "" ]; then
-  export HTTPFS_SSL_ENABLED="false"
-  print "Setting HTTPFS_SSL_ENABLED: ${HTTPFS_SSL_ENABLED}"
-else
-  print "Using   HTTPFS_SSL_ENABLED: ${HTTPFS_SSL_ENABLED}"
-fi
-
-if [ "${HTTPFS_SSL_KEYSTORE_FILE}" = "" ]; then
-  export HTTPFS_SSL_KEYSTORE_FILE=${HOME}/.keystore
-  print "Setting HTTPFS_SSL_KEYSTORE_FILE:     ${HTTPFS_SSL_KEYSTORE_FILE}"
-else
-  print "Using   HTTPFS_SSL_KEYSTORE_FILE:     ${HTTPFS_SSL_KEYSTORE_FILE}"
-fi
-
-if [ "${HTTPFS_SSL_KEYSTORE_PASS}" = "" ]; then
-  export HTTPFS_SSL_KEYSTORE_PASS=password
-  print "Setting HTTPFS_SSL_KEYSTORE_PASS:     ${HTTPFS_SSL_KEYSTORE_PASS}"
-else
-  print "Using   HTTPFS_SSL_KEYSTORE_PASS:     ${HTTPFS_SSL_KEYSTORE_PASS}"
-fi
-
-if [ "${CATALINA_BASE}" = "" ]; then
-  export CATALINA_BASE=${HTTPFS_HOME}/share/hadoop/httpfs/tomcat
-  print "Setting CATALINA_BASE:       ${CATALINA_BASE}"
-else
-  print "Using   CATALINA_BASE:       ${CATALINA_BASE}"
-fi
-
-if [ "${HTTPFS_CATALINA_HOME}" = "" ]; then
-  export HTTPFS_CATALINA_HOME=${CATALINA_BASE}
-  print "Setting HTTPFS_CATALINA_HOME:       ${HTTPFS_CATALINA_HOME}"
-else
-  print "Using   HTTPFS_CATALINA_HOME:       ${HTTPFS_CATALINA_HOME}"
-fi
-
-if [ "${CATALINA_OUT}" = "" ]; then
-  export CATALINA_OUT=${HTTPFS_LOG}/httpfs-catalina.out
-  print "Setting CATALINA_OUT:        ${CATALINA_OUT}"
-else
-  print "Using   CATALINA_OUT:        ${CATALINA_OUT}"
-fi
+  if [[ -n "${HADOOP_SHELL_SCRIPT_DEBUG}" ]]; then
+    varlist=$(env | egrep '(^HTTPFS|^CATALINA)' | cut -f1 -d= | grep -v _PASS)
+    for i in ${varlist}; do
+      hadoop_debug "Setting ${i} to ${!i}"
+    done
+  fi
+}
 
-if [ "${CATALINA_PID}" = "" ]; then
-  export CATALINA_PID=/tmp/httpfs.pid
-  print "Setting CATALINA_PID:        ${CATALINA_PID}"
+if [[ -n "${HADOOP_COMMON_HOME}" ]] &&
+   [[ -e "${HADOOP_COMMON_HOME}/libexec/hadoop-config.sh" ]]; then
+  . "${HADOOP_COMMON_HOME}/libexec/hadoop-config.sh"
+elif [[ -e "${HADOOP_LIBEXEC_DIR}/hadoop-config.sh" ]]; then
+  . "${HADOOP_LIBEXEC_DIR}/hadoop-config.sh"
+elif [[ -e "${HADOOP_PREFIX}/libexec/hadoop-config.sh" ]]; then
+  . "${HADOOP_PREFIX}/libexec/hadoop-config.sh"
 else
-  print "Using   CATALINA_PID:        ${CATALINA_PID}"
+  echo "ERROR: Hadoop common not found." 2>&1
+  exit 1
 fi
-
-print

http://git-wip-us.apache.org/repos/asf/hadoop/blob/21e9e91c/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/sbin/httpfs.sh
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/sbin/httpfs.sh b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/sbin/httpfs.sh
index 65903dc..72dca93 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/sbin/httpfs.sh
+++ b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/sbin/httpfs.sh
@@ -1,4 +1,4 @@
-#!/usr/bin/env bash
+#!/bin/bash
 #
 # Licensed under the Apache License, Version 2.0 (the "License");
 # you may not use this file except in compliance with the License.
@@ -13,53 +13,99 @@
 #  limitations under the License.
 #
 
-# resolve links - $0 may be a softlink
-PRG="${0}"
+function hadoop_usage()
+{
+  echo "Usage: httpfs.sh [--config confdir] [--debug] --daemon start|status|stop"
+  echo "       httpfs.sh [--config confdir] [--debug] COMMAND"
+  echo "            where COMMAND is one of:"
+  echo "  run               Start httpfs in the current window"
+  echo "  run -security     Start in the current window with security manager"
+  echo "  start             Start httpfs in a separate window"
+  echo "  start -security   Start in a separate window with security manager"
+  echo "  status            Return the LSB compliant status"
+  echo "  stop              Stop httpfs, waiting up to 5 seconds for the process to end"
+  echo "  stop n            Stop httpfs, waiting up to n seconds for the process to end"
+  echo "  stop -force       Stop httpfs, wait up to 5 seconds and then use kill -KILL if still running"
+  echo "  stop n -force     Stop httpfs, wait up to n seconds and then use kill -KILL if still running"
+}
 
-while [ -h "${PRG}" ]; do
-  ls=`ls -ld "${PRG}"`
-  link=`expr "$ls" : '.*-> \(.*\)$'`
-  if expr "$link" : '/.*' > /dev/null; then
-    PRG="$link"
-  else
-    PRG=`dirname "${PRG}"`/"$link"
-  fi
-done
-
-BASEDIR=`dirname ${PRG}`
-BASEDIR=`cd ${BASEDIR}/..;pwd`
+# let's locate libexec...
+if [[ -n "${HADOOP_PREFIX}" ]]; then
+  DEFAULT_LIBEXEC_DIR="${HADOOP_PREFIX}/libexec"
+else
+  this="${BASH_SOURCE-$0}"
+  bin=$(cd -P -- "$(dirname -- "${this}")" >/dev/null && pwd -P)
+  DEFAULT_LIBEXEC_DIR="${bin}/../libexec"
+fi
 
-source ${HADOOP_LIBEXEC_DIR:-${BASEDIR}/libexec}/httpfs-config.sh
+HADOOP_LIBEXEC_DIR="${HADOOP_LIBEXEC_DIR:-$DEFAULT_LIBEXEC_DIR}"
+# shellcheck disable=SC2034
+HADOOP_NEW_CONFIG=true
+if [[ -f "${HADOOP_LIBEXEC_DIR}/httpfs-config.sh" ]]; then
+  . "${HADOOP_LIBEXEC_DIR}/httpfs-config.sh"
+else
+  echo "ERROR: Cannot execute ${HADOOP_LIBEXEC_DIR}/httpfs-config.sh." 2>&1
+  exit 1
+fi
 
-# The Java System property 'httpfs.http.port' it is not used by HttpFS,
+# The Java System property 'httpfs.http.port' it is not used by Kms,
 # it is used in Tomcat's server.xml configuration file
 #
-print "Using   CATALINA_OPTS:       ${CATALINA_OPTS}"
 
-catalina_opts="-Dhttpfs.home.dir=${HTTPFS_HOME}";
-catalina_opts="${catalina_opts} -Dhttpfs.config.dir=${HTTPFS_CONFIG}";
-catalina_opts="${catalina_opts} -Dhttpfs.log.dir=${HTTPFS_LOG}";
-catalina_opts="${catalina_opts} -Dhttpfs.temp.dir=${HTTPFS_TEMP}";
-catalina_opts="${catalina_opts} -Dhttpfs.admin.port=${HTTPFS_ADMIN_PORT}";
-catalina_opts="${catalina_opts} -Dhttpfs.http.port=${HTTPFS_HTTP_PORT}";
-catalina_opts="${catalina_opts} -Dhttpfs.http.hostname=${HTTPFS_HTTP_HOSTNAME}";
-catalina_opts="${catalina_opts} -Dhttpfs.ssl.enabled=${HTTPFS_SSL_ENABLED}";
-catalina_opts="${catalina_opts} -Dhttpfs.ssl.keystore.file=${HTTPFS_SSL_KEYSTORE_FILE}";
-catalina_opts="${catalina_opts} -Dhttpfs.ssl.keystore.pass=${HTTPFS_SSL_KEYSTORE_PASS}";
+# Mask the trustStorePassword
+# shellcheck disable=SC2086
+CATALINA_OPTS_DISP="$(echo ${CATALINA_OPTS} | sed -e 's/trustStorePassword=[^ ]*/trustStorePassword=***/')"
 
-print "Adding to CATALINA_OPTS:     ${catalina_opts}"
+hadoop_debug "Using   CATALINA_OPTS:       ${CATALINA_OPTS_DISP}"
 
-export CATALINA_OPTS="${CATALINA_OPTS} ${catalina_opts}"
+# We're using hadoop-common, so set up some stuff it might need:
+hadoop_finalize
+
+hadoop_verify_logdir
+
+if [[ $# = 0 ]]; then
+  case "${HADOOP_DAEMON_MODE}" in
+    status)
+      hadoop_status_daemon "${CATALINA_PID}"
+      exit
+    ;;
+    start)
+      set -- "start"
+    ;;
+    stop)
+      set -- "stop"
+    ;;
+  esac
+fi
+
+hadoop_finalize_catalina_opts
+export CATALINA_OPTS
 
 # A bug in catalina.sh script does not use CATALINA_OPTS for stopping the server
 #
-if [ "${1}" = "stop" ]; then
+if [[ "${1}" = "stop" ]]; then
   export JAVA_OPTS=${CATALINA_OPTS}
 fi
 
-if [ "${HTTPFS_SILENT}" != "true" ]; then
-  exec ${HTTPFS_CATALINA_HOME}/bin/catalina.sh "$@"
-else
-  exec ${HTTPFS_CATALINA_HOME}/bin/catalina.sh "$@" > /dev/null
+# If ssl, the populate the passwords into ssl-server.xml before starting tomcat
+#
+# HTTPFS_SSL_KEYSTORE_PASS is a bit odd.
+# if undefined, then the if test will not enable ssl on its own
+# if "", set it to "password".
+# if custom, use provided password
+#
+if [[ -f "${HADOOP_CATALINA_HOME}/conf/ssl-server.xml.conf" ]]; then
+  if [[ -n "${HTTPFS_SSL_KEYSTORE_PASS+x}" ]] || [[ -n "${HTTPFS_SSL_TRUSTSTORE_PASS}" ]]; then
+    export HTTPFS_SSL_KEYSTORE_PASS=${HTTPFS_SSL_KEYSTORE_PASS:-password}
+    sed -e 's/_httpfs_ssl_keystore_pass_/'${HTTPFS_SSL_KEYSTORE_PASS}'/g' \
+        -e 's/_httpfs_ssl_truststore_pass_/'${HTTPFS_SSL_TRUSTSTORE_PASS}'/g' \
+      "${HADOOP_CATALINA_HOME}/conf/ssl-server.xml.conf" \
+      > "${HADOOP_CATALINA_HOME}/conf/ssl-server.xml"
+    chmod 700 "${HADOOP_CATALINA_HOME}/conf/ssl-server.xml" >/dev/null 2>&1
+  fi
 fi
 
+hadoop_add_param CATALINA_OPTS -Dhttpfs.http.hostname "-Dhttpfs.http.hostname=${HTTPFS_HOST_NAME}"
+hadoop_add_param CATALINA_OPTS -Dhttpfs.ssl.enabled "-Dhttpfs.ssl.enabled=${HTTPFS_SSL_ENABLED}"
+
+exec "${HADOOP_CATALINA_HOME}/bin/catalina.sh" "$@"

http://git-wip-us.apache.org/repos/asf/hadoop/blob/21e9e91c/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/tomcat/ssl-server.xml
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/tomcat/ssl-server.xml b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/tomcat/ssl-server.xml
deleted file mode 100644
index c91c2e2..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/tomcat/ssl-server.xml
+++ /dev/null
@@ -1,135 +0,0 @@
-<?xml version='1.0' encoding='utf-8'?>
-<!--
-
-   All Rights Reserved.
-
-  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.
--->
-<!-- Note:  A "Server" is not itself a "Container", so you may not
-     define subcomponents such as "Valves" at this level.
-     Documentation at /docs/config/server.html
- -->
-<Server port="${httpfs.admin.port}" shutdown="SHUTDOWN">
-
-  <!--APR library loader. Documentation at /docs/apr.html -->
-  <Listener className="org.apache.catalina.core.AprLifecycleListener"
-            SSLEngine="on"/>
-  <!--Initialize Jasper prior to webapps are loaded. Documentation at /docs/jasper-howto.html -->
-  <Listener className="org.apache.catalina.core.JasperListener"/>
-  <!-- Prevent memory leaks due to use of particular java/javax APIs-->
-  <Listener
-    className="org.apache.catalina.core.JreMemoryLeakPreventionListener"/>
-  <!-- JMX Support for the Tomcat server. Documentation at /docs/non-existent.html -->
-  <Listener className="org.apache.catalina.mbeans.ServerLifecycleListener"/>
-  <Listener
-    className="org.apache.catalina.mbeans.GlobalResourcesLifecycleListener"/>
-
-  <!-- Global JNDI resources
-       Documentation at /docs/jndi-resources-howto.html
-  -->
-  <GlobalNamingResources>
-    <!-- Editable user database that can also be used by
-         UserDatabaseRealm to authenticate users
-    -->
-    <Resource name="UserDatabase" auth="Container"
-              type="org.apache.catalina.UserDatabase"
-              description="User database that can be updated and saved"
-              factory="org.apache.catalina.users.MemoryUserDatabaseFactory"
-              pathname="conf/tomcat-users.xml"/>
-  </GlobalNamingResources>
-
-  <!-- A "Service" is a collection of one or more "Connectors" that share
-       a single "Container" Note:  A "Service" is not itself a "Container",
-       so you may not define subcomponents such as "Valves" at this level.
-       Documentation at /docs/config/service.html
-   -->
-  <Service name="Catalina">
-
-    <!--The connectors can use a shared executor, you can define one or more named thread pools-->
-    <!--
-    <Executor name="tomcatThreadPool" namePrefix="catalina-exec-"
-        maxThreads="150" minSpareThreads="4"/>
-    -->
-
-    <!-- Define a SSL HTTP/1.1 Connector on port 8443
-         This connector uses the JSSE configuration, when using APR, the
-         connector should be using the OpenSSL style configuration
-         described in the APR documentation -->
-    <Connector port="${httpfs.http.port}" protocol="HTTP/1.1" SSLEnabled="true"
-               maxThreads="150" scheme="https" secure="true"
-               clientAuth="false" sslEnabledProtocols="TLSv1,SSLv2Hello"
-               keystoreFile="${httpfs.ssl.keystore.file}"
-               keystorePass="${httpfs.ssl.keystore.pass}"/>
-
-    <!-- Define an AJP 1.3 Connector on port 8009 -->
-
-
-    <!-- An Engine represents the entry point (within Catalina) that processes
- every request.  The Engine implementation for Tomcat stand alone
- analyzes the HTTP headers included with the request, and passes them
- on to the appropriate Host (virtual host).
- Documentation at /docs/config/engine.html -->
-
-    <!-- You should set jvmRoute to support load-balancing via AJP ie :
-    <Engine name="Catalina" defaultHost="localhost" jvmRoute="jvm1">
-    -->
-    <Engine name="Catalina" defaultHost="localhost">
-
-      <!--For clustering, please take a look at documentation at:
-          /docs/cluster-howto.html  (simple how to)
-          /docs/config/cluster.html (reference documentation) -->
-      <!--
-      <Cluster className="org.apache.catalina.ha.tcp.SimpleTcpCluster"/>
-      -->
-
-      <!-- The request dumper valve dumps useful debugging information about
-           the request and response data received and sent by Tomcat.
-           Documentation at: /docs/config/valve.html -->
-      <!--
-      <Valve className="org.apache.catalina.valves.RequestDumperValve"/>
-      -->
-
-      <!-- This Realm uses the UserDatabase configured in the global JNDI
-           resources under the key "UserDatabase".  Any edits
-           that are performed against this UserDatabase are immediately
-           available for use by the Realm.  -->
-      <Realm className="org.apache.catalina.realm.UserDatabaseRealm"
-             resourceName="UserDatabase"/>
-
-      <!-- Define the default virtual host
-           Note: XML Schema validation will not work with Xerces 2.2.
-       -->
-      <Host name="localhost" appBase="webapps"
-            unpackWARs="true" autoDeploy="true"
-            xmlValidation="false" xmlNamespaceAware="false">
-
-        <!-- SingleSignOn valve, share authentication between web applications
-             Documentation at: /docs/config/valve.html -->
-        <!--
-        <Valve className="org.apache.catalina.authenticator.SingleSignOn" />
-        -->
-
-        <!-- Access log processes all example.
-             Documentation at: /docs/config/valve.html -->
-        <!--
-        <Valve className="org.apache.catalina.valves.AccessLogValve" directory="logs"
-               prefix="localhost_access_log." suffix=".txt" pattern="common" resolveHosts="false"/>
-        -->
-
-      </Host>
-    </Engine>
-  </Service>
-</Server>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/21e9e91c/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/tomcat/ssl-server.xml.conf
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/tomcat/ssl-server.xml.conf b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/tomcat/ssl-server.xml.conf
new file mode 100644
index 0000000..4a90532
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/tomcat/ssl-server.xml.conf
@@ -0,0 +1,135 @@
+<?xml version='1.0' encoding='utf-8'?>
+<!--
+
+   All Rights Reserved.
+
+  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.
+-->
+<!-- Note:  A "Server" is not itself a "Container", so you may not
+     define subcomponents such as "Valves" at this level.
+     Documentation at /docs/config/server.html
+ -->
+<Server port="${httpfs.admin.port}" shutdown="SHUTDOWN">
+
+  <!--APR library loader. Documentation at /docs/apr.html -->
+  <Listener className="org.apache.catalina.core.AprLifecycleListener"
+            SSLEngine="on"/>
+  <!--Initialize Jasper prior to webapps are loaded. Documentation at /docs/jasper-howto.html -->
+  <Listener className="org.apache.catalina.core.JasperListener"/>
+  <!-- Prevent memory leaks due to use of particular java/javax APIs-->
+  <Listener
+    className="org.apache.catalina.core.JreMemoryLeakPreventionListener"/>
+  <!-- JMX Support for the Tomcat server. Documentation at /docs/non-existent.html -->
+  <Listener className="org.apache.catalina.mbeans.ServerLifecycleListener"/>
+  <Listener
+    className="org.apache.catalina.mbeans.GlobalResourcesLifecycleListener"/>
+
+  <!-- Global JNDI resources
+       Documentation at /docs/jndi-resources-howto.html
+  -->
+  <GlobalNamingResources>
+    <!-- Editable user database that can also be used by
+         UserDatabaseRealm to authenticate users
+    -->
+    <Resource name="UserDatabase" auth="Container"
+              type="org.apache.catalina.UserDatabase"
+              description="User database that can be updated and saved"
+              factory="org.apache.catalina.users.MemoryUserDatabaseFactory"
+              pathname="conf/tomcat-users.xml"/>
+  </GlobalNamingResources>
+
+  <!-- A "Service" is a collection of one or more "Connectors" that share
+       a single "Container" Note:  A "Service" is not itself a "Container",
+       so you may not define subcomponents such as "Valves" at this level.
+       Documentation at /docs/config/service.html
+   -->
+  <Service name="Catalina">
+
+    <!--The connectors can use a shared executor, you can define one or more named thread pools-->
+    <!--
+    <Executor name="tomcatThreadPool" namePrefix="catalina-exec-"
+        maxThreads="httpfs.max.threads" minSpareThreads="4"/>
+    -->
+
+    <!-- Define a SSL HTTP/1.1 Connector on port 8443
+         This connector uses the JSSE configuration, when using APR, the
+         connector should be using the OpenSSL style configuration
+         described in the APR documentation -->
+    <Connector port="${httpfs.http.port}" protocol="HTTP/1.1" SSLEnabled="true"
+               maxThreads="150" scheme="https" secure="true"
+               clientAuth="false" sslEnabledProtocols="TLSv1,SSLv2Hello"
+               keystoreFile="${httpfs.ssl.keystore.file}"
+               keystorePass="_httpfs_ssl_keystore_pass_"/>
+
+    <!-- Define an AJP 1.3 Connector on port 8009 -->
+
+
+    <!-- An Engine represents the entry point (within Catalina) that processes
+ every request.  The Engine implementation for Tomcat stand alone
+ analyzes the HTTP headers included with the request, and passes them
+ on to the appropriate Host (virtual host).
+ Documentation at /docs/config/engine.html -->
+
+    <!-- You should set jvmRoute to support load-balancing via AJP ie :
+    <Engine name="Catalina" defaultHost="localhost" jvmRoute="jvm1">
+    -->
+    <Engine name="Catalina" defaultHost="localhost">
+
+      <!--For clustering, please take a look at documentation at:
+          /docs/cluster-howto.html  (simple how to)
+          /docs/config/cluster.html (reference documentation) -->
+      <!--
+      <Cluster className="org.apache.catalina.ha.tcp.SimpleTcpCluster"/>
+      -->
+
+      <!-- The request dumper valve dumps useful debugging information about
+           the request and response data received and sent by Tomcat.
+           Documentation at: /docs/config/valve.html -->
+      <!--
+      <Valve className="org.apache.catalina.valves.RequestDumperValve"/>
+      -->
+
+      <!-- This Realm uses the UserDatabase configured in the global JNDI
+           resources under the key "UserDatabase".  Any edits
+           that are performed against this UserDatabase are immediately
+           available for use by the Realm.  -->
+      <Realm className="org.apache.catalina.realm.UserDatabaseRealm"
+             resourceName="UserDatabase"/>
+
+      <!-- Define the default virtual host
+           Note: XML Schema validation will not work with Xerces 2.2.
+       -->
+      <Host name="localhost" appBase="webapps"
+            unpackWARs="true" autoDeploy="true"
+            xmlValidation="false" xmlNamespaceAware="false">
+
+        <!-- SingleSignOn valve, share authentication between web applications
+             Documentation at: /docs/config/valve.html -->
+        <!--
+        <Valve className="org.apache.catalina.authenticator.SingleSignOn" />
+        -->
+
+        <!-- Access log processes all example.
+             Documentation at: /docs/config/valve.html -->
+        <!--
+        <Valve className="org.apache.catalina.valves.AccessLogValve" directory="logs"
+               prefix="localhost_access_log." suffix=".txt" pattern="common" resolveHosts="false"/>
+        -->
+
+      </Host>
+    </Engine>
+  </Service>
+</Server>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/21e9e91c/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/site/apt/ServerSetup.apt.vm
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/site/apt/ServerSetup.apt.vm b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/site/apt/ServerSetup.apt.vm
deleted file mode 100644
index 878ab1f..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/site/apt/ServerSetup.apt.vm
+++ /dev/null
@@ -1,159 +0,0 @@
-~~ 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.
-
-  ---
-  Hadoop HDFS over HTTP ${project.version} - Server Setup
-  ---
-  ---
-  ${maven.build.timestamp}
-
-Hadoop HDFS over HTTP ${project.version} - Server Setup
-
-  This page explains how to quickly setup HttpFS with Pseudo authentication
-  against a Hadoop cluster with Pseudo authentication.
-
-* Requirements
-
-    * Java 6+
-
-    * Maven 3+
-
-* Install HttpFS
-
-+---+
-~ $ tar xzf  httpfs-${project.version}.tar.gz
-+---+
-
-* Configure HttpFS
-
-  By default, HttpFS assumes that Hadoop configuration files
-  (<<<core-site.xml & hdfs-site.xml>>>) are in the HttpFS
-  configuration directory.
-
-  If this is not the case, add to the <<<httpfs-site.xml>>> file the
-  <<<httpfs.hadoop.config.dir>>> property set to the location
-  of the Hadoop configuration directory.
-
-* Configure Hadoop
-
-  Edit Hadoop <<<core-site.xml>>> and defined the Unix user that will
-  run the HttpFS server as a proxyuser. For example:
-
-+---+
-  ...
-  <property>
-    <name>hadoop.proxyuser.#HTTPFSUSER#.hosts</name>
-    <value>httpfs-host.foo.com</value>
-  </property>
-  <property>
-    <name>hadoop.proxyuser.#HTTPFSUSER#.groups</name>
-    <value>*</value>
-  </property>
-  ...
-+---+
-
-  IMPORTANT: Replace <<<#HTTPFSUSER#>>> with the Unix user that will
-  start the HttpFS server.
-
-* Restart Hadoop
-
-  You need to restart Hadoop for the proxyuser configuration ot become
-  active.
-
-* Start/Stop HttpFS
-
-  To start/stop HttpFS use HttpFS's bin/httpfs.sh script. For example:
-
-+---+
-httpfs-${project.version} $ bin/httpfs.sh start
-+---+
-
-  NOTE: Invoking the script without any parameters list all possible
-  parameters (start, stop, run, etc.). The <<<httpfs.sh>>> script is a wrapper
-  for Tomcat's <<<catalina.sh>>> script that sets the environment variables
-  and Java System properties required to run HttpFS server.
-
-* Test HttpFS is working
-
-+---+
-~ $ curl -i "http://<HTTPFSHOSTNAME>:14000?user.name=babu&op=homedir"
-HTTP/1.1 200 OK
-Content-Type: application/json
-Transfer-Encoding: chunked
-
-{"homeDir":"http:\/\/<HTTPFS_HOST>:14000\/user\/babu"}
-+---+
-
-* Embedded Tomcat Configuration
-
-  To configure the embedded Tomcat go to the <<<tomcat/conf>>>.
-
-  HttpFS preconfigures the HTTP and Admin ports in Tomcat's <<<server.xml>>> to
-  14000 and 14001.
-
-  Tomcat logs are also preconfigured to go to HttpFS's <<<logs/>>> directory.
-
-  The following environment variables (which can be set in HttpFS's
-  <<<conf/httpfs-env.sh>>> script) can be used to alter those values:
-
-  * HTTPFS_HTTP_PORT
-
-  * HTTPFS_ADMIN_PORT
-
-  * HTTPFS_LOG
-
-* HttpFS Configuration
-
-  HttpFS supports the following {{{./httpfs-default.html}configuration properties}}
-  in the HttpFS's <<<conf/httpfs-site.xml>>> configuration file.
-
-* HttpFS over HTTPS (SSL)
-
-  To configure HttpFS to work over SSL edit the {{httpfs-env.sh}} script in the
-  configuration directory setting the {{HTTPFS_SSL_ENABLED}} to {{true}}.
-
-  In addition, the following 2 properties may be defined (shown with default
-  values):
-
-    * HTTPFS_SSL_KEYSTORE_FILE=${HOME}/.keystore
-
-    * HTTPFS_SSL_KEYSTORE_PASS=password
-
-  In the HttpFS <<<tomcat/conf>>> directory, replace the <<<server.xml>>> file
-  with the  <<<ssl-server.xml>>> file.
-
-
-  You need to create an SSL certificate for the HttpFS server. As the
-  <<<httpfs>>> Unix user, using the Java <<<keytool>>> command to create the
-  SSL certificate:
-
-+---+
-$ keytool -genkey -alias tomcat -keyalg RSA
-+---+
-
-  You will be asked a series of questions in an interactive prompt.  It will
-  create the keystore file, which will be named <<.keystore>> and located in the
-  <<<httpfs>>> user home directory.
-
-  The password you enter for "keystore password" must match the  value of the
-  <<<HTTPFS_SSL_KEYSTORE_PASS>>> environment variable set in the
-  <<<httpfs-env.sh>>> script in the configuration directory.
-
-  The answer to "What is your first and last name?" (i.e. "CN") must be the
-  hostname of the machine where the HttpFS Server will be running.
-
-  Start HttpFS. It should work over HTTPS.
-
-  Using the Hadoop <<<FileSystem>>> API or the Hadoop FS shell, use the
-  <<<swebhdfs://>>> scheme. Make sure the JVM is picking up the truststore
-  containing the public key of the SSL certificate if using a self-signed
-  certificate.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/21e9e91c/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/site/apt/UsingHttpTools.apt.vm
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/site/apt/UsingHttpTools.apt.vm b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/site/apt/UsingHttpTools.apt.vm
deleted file mode 100644
index c93e20b..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/site/apt/UsingHttpTools.apt.vm
+++ /dev/null
@@ -1,87 +0,0 @@
-~~ 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.
-
-  ---
-  Hadoop HDFS over HTTP ${project.version} - Using HTTP Tools
-  ---
-  ---
-  ${maven.build.timestamp}
-
-Hadoop HDFS over HTTP ${project.version} - Using HTTP Tools
-
-* Security
-
-  Out of the box HttpFS supports both pseudo authentication and Kerberos HTTP
-  SPNEGO authentication.
-
-** Pseudo Authentication
-
-  With pseudo authentication the user name must be specified in the
-  <<<user.name=\<USERNAME\>>>> query string parameter of a HttpFS URL.
-  For example:
-
-+---+
-$ curl "http://<HTTFS_HOST>:14000/webhdfs/v1?op=homedir&user.name=babu"
-+---+
-
-** Kerberos HTTP SPNEGO Authentication
-
-  Kerberos HTTP SPNEGO authentication requires a tool or library supporting
-  Kerberos HTTP SPNEGO protocol.
-
-  IMPORTANT: If using <<<curl>>>, the <<<curl>>> version being used must support
-  GSS (<<<curl -V>>> prints out 'GSS' if it supports it).
-
-  For example:
-
-+---+
-$ kinit
-Please enter the password for tucu@LOCALHOST:
-$ curl --negotiate -u foo "http://<HTTPFS_HOST>:14000/webhdfs/v1?op=homedir"
-Enter host password for user 'foo':
-+---+
-
-  NOTE: the <<<-u USER>>> option is required by the <<<--negotiate>>> but it is
-  not used. Use any value as <<<USER>>> and when asked for the password press
-  [ENTER] as the password value is ignored.
-
-** {Remembering Who I Am} (Establishing an Authenticated Session)
-
-  As most authentication mechanisms, Hadoop HTTP authentication authenticates
-  users once and issues a short-lived authentication token to be presented in
-  subsequent requests. This authentication token is a signed HTTP Cookie.
-
-  When using tools like <<<curl>>>, the authentication token must be stored on
-  the first request doing authentication, and submitted in subsequent requests.
-  To do this with curl the <<<-b>>> and <<<-c>>> options to save and send HTTP
-  Cookies must be used.
-
-  For example, the first request doing authentication should save the received
-  HTTP Cookies.
-
-    Using Pseudo Authentication:
-
-+---+
-$ curl -c ~/.httpfsauth "http://<HTTPFS_HOST>:14000/webhdfs/v1?op=homedir&user.name=babu"
-+---+
-
-    Using Kerberos HTTP SPNEGO authentication:
-
-+---+
-$ curl --negotiate -u foo -c ~/.httpfsauth "http://<HTTPFS_HOST>:14000/webhdfs/v1?op=homedir"
-+---+
-
-  Then, subsequent requests forward the previously received HTTP Cookie:
-
-+---+
-$ curl -b ~/.httpfsauth "http://<HTTPFS_HOST>:14000/webhdfs/v1?op=liststatus"
-+---+

http://git-wip-us.apache.org/repos/asf/hadoop/blob/21e9e91c/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/site/apt/index.apt.vm
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/site/apt/index.apt.vm b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/site/apt/index.apt.vm
deleted file mode 100644
index f51e743..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/site/apt/index.apt.vm
+++ /dev/null
@@ -1,83 +0,0 @@
-~~ 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.
-
-  ---
-  Hadoop HDFS over HTTP - Documentation Sets ${project.version}
-  ---
-  ---
-  ${maven.build.timestamp}
-
-Hadoop HDFS over HTTP - Documentation Sets ${project.version}
-
-  HttpFS is a server that provides a REST HTTP gateway supporting all HDFS
-  File System operations (read and write). And it is inteoperable with the
-  <<webhdfs>> REST HTTP API.
-
-  HttpFS can be used to transfer data between clusters running different
-  versions of Hadoop (overcoming RPC versioning issues), for example using
-  Hadoop DistCP.
-
-  HttpFS can be used to access data in HDFS on a cluster behind of a firewall
-  (the HttpFS server acts as a gateway and is the only system that is allowed
-  to cross the firewall into the cluster).
-
-  HttpFS can be used to access data in HDFS using HTTP utilities (such as curl
-  and wget) and HTTP libraries Perl from other languages than Java.
-
-  The <<webhdfs>> client FileSytem implementation can be used to access HttpFS
-  using the Hadoop filesystem command (<<<hadoop fs>>>) line tool as well as
-  from Java aplications using the Hadoop FileSystem Java API.
-
-  HttpFS has built-in security supporting Hadoop pseudo authentication and
-  HTTP SPNEGO Kerberos and other pluggable authentication mechanims. It also
-  provides Hadoop proxy user support.
-
-* How Does HttpFS Works?
-
-  HttpFS is a separate service from Hadoop NameNode.
-
-  HttpFS itself is Java web-application and it runs using a preconfigured Tomcat
-  bundled with HttpFS binary distribution.
-
-  HttpFS HTTP web-service API calls are HTTP REST calls that map to a HDFS file
-  system operation. For example, using the <<<curl>>> Unix command:
-
-  * <<<$ curl http://httpfs-host:14000/webhdfs/v1/user/foo/README.txt>>> returns
-  the contents of the HDFS <<</user/foo/README.txt>>> file.
-
-  * <<<$ curl http://httpfs-host:14000/webhdfs/v1/user/foo?op=list>>> returns the
-  contents of the HDFS <<</user/foo>>> directory in JSON format.
-
-  * <<<$ curl -X POST http://httpfs-host:14000/webhdfs/v1/user/foo/bar?op=mkdirs>>>
-  creates the HDFS <<</user/foo.bar>>> directory.
-
-* How HttpFS and Hadoop HDFS Proxy differ?
-
-  HttpFS was inspired by Hadoop HDFS proxy.
-
-  HttpFS can be seen as a full rewrite of Hadoop HDFS proxy.
-
-  Hadoop HDFS proxy provides a subset of file system operations (read only),
-  HttpFS provides support for all file system operations.
-
-  HttpFS uses a clean HTTP REST API making its use with HTTP tools more
-  intuitive.
-
-  HttpFS supports Hadoop pseudo authentication, Kerberos SPNEGOS authentication
-  and Hadoop proxy users. Hadoop HDFS proxy did not.
-
-* User and Developer Documentation
-
-  * {{{./ServerSetup.html}HttpFS Server Setup}}
-
-  * {{{./UsingHttpTools.html}Using HTTP Tools}}
-

http://git-wip-us.apache.org/repos/asf/hadoop/blob/21e9e91c/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/site/markdown/ServerSetup.md.vm
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/site/markdown/ServerSetup.md.vm b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/site/markdown/ServerSetup.md.vm
new file mode 100644
index 0000000..3c7f9d3
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/site/markdown/ServerSetup.md.vm
@@ -0,0 +1,121 @@
+<!---
+  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 HDFS over HTTP - Server Setup
+====================================
+
+This page explains how to quickly setup HttpFS with Pseudo authentication against a Hadoop cluster with Pseudo authentication.
+
+Install HttpFS
+--------------
+
+    ~ $ tar xzf  httpfs-${project.version}.tar.gz
+
+Configure HttpFS
+----------------
+
+By default, HttpFS assumes that Hadoop configuration files (`core-site.xml & hdfs-site.xml`) are in the HttpFS configuration directory.
+
+If this is not the case, add to the `httpfs-site.xml` file the `httpfs.hadoop.config.dir` property set to the location of the Hadoop configuration directory.
+
+Configure Hadoop
+----------------
+
+Edit Hadoop `core-site.xml` and defined the Unix user that will run the HttpFS server as a proxyuser. For example:
+
+```xml
+  <property>
+    <name>hadoop.proxyuser.#HTTPFSUSER#.hosts</name>
+    <value>httpfs-host.foo.com</value>
+  </property>
+  <property>
+    <name>hadoop.proxyuser.#HTTPFSUSER#.groups</name>
+    <value>*</value>
+  </property>
+```
+
+IMPORTANT: Replace `#HTTPFSUSER#` with the Unix user that will start the HttpFS server.
+
+Restart Hadoop
+--------------
+
+You need to restart Hadoop for the proxyuser configuration ot become active.
+
+Start/Stop HttpFS
+-----------------
+
+To start/stop HttpFS use HttpFS's sbin/httpfs.sh script. For example:
+
+    $ sbin/httpfs.sh start
+
+NOTE: Invoking the script without any parameters list all possible parameters (start, stop, run, etc.). The `httpfs.sh` script is a wrapper for Tomcat's `catalina.sh` script that sets the environment variables and Java System properties required to run HttpFS server.
+
+Test HttpFS is working
+----------------------
+
+    ~ $ curl -i "http://<HTTPFSHOSTNAME>:14000?user.name=babu&op=homedir"
+    HTTP/1.1 200 OK
+    Content-Type: application/json
+    Transfer-Encoding: chunked
+
+    {"homeDir":"http:\/\/<HTTPFS_HOST>:14000\/user\/babu"}
+
+Embedded Tomcat Configuration
+-----------------------------
+
+To configure the embedded Tomcat go to the `tomcat/conf`.
+
+HttpFS preconfigures the HTTP and Admin ports in Tomcat's `server.xml` to 14000 and 14001.
+
+Tomcat logs are also preconfigured to go to HttpFS's `logs/` directory.
+
+The following environment variables (which can be set in HttpFS's `etc/hadoop/httpfs-env.sh` script) can be used to alter those values:
+
+* HTTPFS\_HTTP\_PORT
+
+* HTTPFS\_ADMIN\_PORT
+
+* HADOOP\_LOG\_DIR
+
+HttpFS Configuration
+--------------------
+
+HttpFS supports the following [configuration properties](./httpfs-default.html) in the HttpFS's `etc/hadoop/httpfs-site.xml` configuration file.
+
+HttpFS over HTTPS (SSL)
+-----------------------
+
+To configure HttpFS to work over SSL edit the [httpfs-env.sh](#httpfs-env.sh) script in the configuration directory setting the [HTTPFS\_SSL\_ENABLED](#HTTPFS_SSL_ENABLED) to [true](#true).
+
+In addition, the following 2 properties may be defined (shown with default values):
+
+* HTTPFS\_SSL\_KEYSTORE\_FILE=$HOME/.keystore
+
+* HTTPFS\_SSL\_KEYSTORE\_PASS=password
+
+In the HttpFS `tomcat/conf` directory, replace the `server.xml` file with the `ssl-server.xml` file.
+
+You need to create an SSL certificate for the HttpFS server. As the `httpfs` Unix user, using the Java `keytool` command to create the SSL certificate:
+
+    $ keytool -genkey -alias tomcat -keyalg RSA
+
+You will be asked a series of questions in an interactive prompt. It will create the keystore file, which will be named **.keystore** and located in the `httpfs` user home directory.
+
+The password you enter for "keystore password" must match the value of the `HTTPFS_SSL_KEYSTORE_PASS` environment variable set in the `httpfs-env.sh` script in the configuration directory.
+
+The answer to "What is your first and last name?" (i.e. "CN") must be the hostname of the machine where the HttpFS Server will be running.
+
+Start HttpFS. It should work over HTTPS.
+
+Using the Hadoop `FileSystem` API or the Hadoop FS shell, use the `swebhdfs://` scheme. Make sure the JVM is picking up the truststore containing the public key of the SSL certificate if using a self-signed certificate.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/21e9e91c/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/site/markdown/UsingHttpTools.md
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/site/markdown/UsingHttpTools.md b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/site/markdown/UsingHttpTools.md
new file mode 100644
index 0000000..3045ad6
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/site/markdown/UsingHttpTools.md
@@ -0,0 +1,62 @@
+<!---
+  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 HDFS over HTTP - Using HTTP Tools
+========================================
+
+Security
+--------
+
+Out of the box HttpFS supports both pseudo authentication and Kerberos HTTP SPNEGO authentication.
+
+### Pseudo Authentication
+
+With pseudo authentication the user name must be specified in the `user.name=<USERNAME>` query string parameter of a HttpFS URL. For example:
+
+    $ curl "http://<HTTFS_HOST>:14000/webhdfs/v1?op=homedir&user.name=babu"
+
+### Kerberos HTTP SPNEGO Authentication
+
+Kerberos HTTP SPNEGO authentication requires a tool or library supporting Kerberos HTTP SPNEGO protocol.
+
+IMPORTANT: If using `curl`, the `curl` version being used must support GSS (`curl -V` prints out 'GSS' if it supports it).
+
+For example:
+
+    $ kinit
+    Please enter the password for user@LOCALHOST:
+    $ curl --negotiate -u foo "http://<HTTPFS_HOST>:14000/webhdfs/v1?op=homedir"
+    Enter host password for user 'foo':
+
+NOTE: the `-u USER` option is required by the `--negotiate` but it is not used. Use any value as `USER` and when asked for the password press [ENTER] as the password value is ignored.
+
+### Remembering Who I Am (Establishing an Authenticated Session)
+
+As most authentication mechanisms, Hadoop HTTP authentication authenticates users once and issues a short-lived authentication token to be presented in subsequent requests. This authentication token is a signed HTTP Cookie.
+
+When using tools like `curl`, the authentication token must be stored on the first request doing authentication, and submitted in subsequent requests. To do this with curl the `-b` and `-c` options to save and send HTTP Cookies must be used.
+
+For example, the first request doing authentication should save the received HTTP Cookies.
+
+Using Pseudo Authentication:
+
+    $ curl -c ~/.httpfsauth "http://<HTTPFS_HOST>:14000/webhdfs/v1?op=homedir&user.name=foo"
+
+Using Kerberos HTTP SPNEGO authentication:
+
+    $ curl --negotiate -u foo -c ~/.httpfsauth "http://<HTTPFS_HOST>:14000/webhdfs/v1?op=homedir"
+
+Then, subsequent requests forward the previously received HTTP Cookie:
+
+    $ curl -b ~/.httpfsauth "http://<HTTPFS_HOST>:14000/webhdfs/v1?op=liststatus"

http://git-wip-us.apache.org/repos/asf/hadoop/blob/21e9e91c/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/site/markdown/index.md
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/site/markdown/index.md b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/site/markdown/index.md
new file mode 100644
index 0000000..ac8f4ca
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/site/markdown/index.md
@@ -0,0 +1,52 @@
+<!---
+  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 HDFS over HTTP - Documentation Sets
+==========================================
+
+HttpFS is a server that provides a REST HTTP gateway supporting all HDFS File System operations (read and write). And it is inteoperable with the **webhdfs** REST HTTP API.
+
+HttpFS can be used to transfer data between clusters running different versions of Hadoop (overcoming RPC versioning issues), for example using Hadoop DistCP.
+
+HttpFS can be used to access data in HDFS on a cluster behind of a firewall (the HttpFS server acts as a gateway and is the only system that is allowed to cross the firewall into the cluster).
+
+HttpFS can be used to access data in HDFS using HTTP utilities (such as curl and wget) and HTTP libraries Perl from other languages than Java.
+
+The **webhdfs** client FileSytem implementation can be used to access HttpFS using the Hadoop filesystem command (`hadoop fs`) line tool as well as from Java aplications using the Hadoop FileSystem Java API.
+
+HttpFS has built-in security supporting Hadoop pseudo authentication and HTTP SPNEGO Kerberos and other pluggable authentication mechanims. It also provides Hadoop proxy user support.
+
+How Does HttpFS Works?
+----------------------
+
+HttpFS is a separate service from Hadoop NameNode.
+
+HttpFS itself is Java web-application and it runs using a preconfigured Tomcat bundled with HttpFS binary distribution.
+
+HttpFS HTTP web-service API calls are HTTP REST calls that map to a HDFS file system operation. For example, using the `curl` Unix command:
+
+* `$ curl http://httpfs-host:14000/webhdfs/v1/user/foo/README.txt` returns the contents of the HDFS `/user/foo/README.txt` file.
+
+* `$ curl http://httpfs-host:14000/webhdfs/v1/user/foo?op=list` returns the contents of the HDFS `/user/foo` directory in JSON format.
+
+* `$ curl -X POST http://httpfs-host:14000/webhdfs/v1/user/foo/bar?op=mkdirs` creates the HDFS `/user/foo.bar` directory.
+
+User and Developer Documentation
+--------------------------------
+
+* [HttpFS Server Setup](./ServerSetup.html)
+
+* [Using HTTP Tools](./UsingHttpTools.html)
+
+

http://git-wip-us.apache.org/repos/asf/hadoop/blob/21e9e91c/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 f8b0c37..6dc7a0f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -141,6 +141,8 @@ Trunk (Unreleased)
     HDFS-7668. Convert site documentation from apt to markdown (Masatake
     Iwasaki via aw)
 
+    HDFS-7460. Rewrite httpfs to use new shell framework (John Smith via aw)
+
   OPTIMIZATIONS
 
   BUG FIXES


[33/50] [abbrv] hadoop git commit: YARN-3168. Convert site documentation from apt to markdown (Gururaj Shetty via aw)

Posted by zh...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/06aca7c6/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/apt/WritingYarnApplications.apt.vm
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/apt/WritingYarnApplications.apt.vm b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/apt/WritingYarnApplications.apt.vm
deleted file mode 100644
index 57a47fd..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/apt/WritingYarnApplications.apt.vm
+++ /dev/null
@@ -1,757 +0,0 @@
-~~ 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 Map Reduce Next Generation-${project.version} - Writing YARN
-  Applications
-  ---
-  ---
-  ${maven.build.timestamp}
-
-Hadoop MapReduce Next Generation - Writing YARN Applications
-
-%{toc|section=1|fromDepth=0}
-
-* Purpose
-
-  This document describes, at a high-level, the way to implement new
-  Applications for YARN.
-
-* Concepts and Flow
-
-  The general concept is that an <application submission client> submits an
-  <application> to the YARN <ResourceManager> (RM). This can be done through
-  setting up a <<<YarnClient>>> object. After <<<YarnClient>>> is started, the
-  client can then set up application context, prepare the very first container of
-  the application that contains the <ApplicationMaster> (AM), and then submit
-  the application. You need to provide information such as the details about the
-  local files/jars that need to be available for your application to run, the
-  actual command that needs to be executed (with the necessary command line
-  arguments), any OS environment settings (optional), etc. Effectively, you
-  need to describe the Unix process(es) that needs to be launched for your
-  ApplicationMaster.
-
-  The YARN ResourceManager will then launch the ApplicationMaster (as
-  specified) on an allocated container. The ApplicationMaster communicates with
-  YARN cluster, and handles application execution. It performs operations in an
-  asynchronous fashion. During application launch time, the main tasks of the
-  ApplicationMaster are: a) communicating with the ResourceManager to negotiate
-  and allocate resources for future containers, and b) after container
-  allocation, communicating YARN <NodeManager>s (NMs) to launch application
-  containers on them. Task a) can be performed asynchronously through an
-  <<<AMRMClientAsync>>> object, with event handling methods specified in a
-  <<<AMRMClientAsync.CallbackHandler>>> type of event handler. The event handler
-  needs to be set to the client explicitly. Task b) can be performed by launching
-  a runnable object that then launches containers when there are containers
-  allocated. As part of launching this container, the AM has to
-  specify the <<<ContainerLaunchContext>>> that has the launch information such as
-  command line specification, environment, etc.
-
-  During the execution of an application, the ApplicationMaster communicates
-  NodeManagers through <<<NMClientAsync>>> object. All container events are
-  handled by <<<NMClientAsync.CallbackHandler>>>, associated with
-  <<<NMClientAsync>>>. A typical callback handler handles client start, stop,
-  status update and error. ApplicationMaster also reports execution progress to
-  ResourceManager by handling the <<<getProgress()>>> method of
-  <<<AMRMClientAsync.CallbackHandler>>>.
-  
-  Other than asynchronous clients, there are synchronous versions for certain
-  workflows (<<<AMRMClient>>> and <<<NMClient>>>). The asynchronous clients are
-  recommended because of (subjectively) simpler usages, and this article
-  will mainly cover the asynchronous clients. Please refer to <<<AMRMClient>>>
-  and <<<NMClient>>> for more information on synchronous clients.
-
-* Interfaces
-
-  The interfaces you'd most like be concerned with are:
-
-  * <<Client>>\<--\><<ResourceManager>>\
-    By using <<<YarnClient>>> objects.
-
-  * <<ApplicationMaster>>\<--\><<ResourceManager>>\
-    By using <<<AMRMClientAsync>>> objects, handling events asynchronously by
-    <<<AMRMClientAsync.CallbackHandler>>>
-
-  * <<ApplicationMaster>>\<--\><<NodeManager>>\
-    Launch containers. Communicate with NodeManagers
-    by using <<<NMClientAsync>>> objects, handling container events by
-    <<<NMClientAsync.CallbackHandler>>>
-
-  []
-
-  <<Note>>
-  
-    * The three main protocols for YARN application (ApplicationClientProtocol,
-      ApplicationMasterProtocol and ContainerManagementProtocol) are still
-      preserved. The 3 clients wrap these 3 protocols to provide simpler
-      programming model for YARN applications.
-    
-    * Under very rare circumstances, programmer may want to directly use the 3
-      protocols to implement an application. However, note that <such behaviors
-      are no longer encouraged for general use cases>.
-
-    []
-
-* Writing a Simple Yarn Application
-
-** Writing a simple Client
-
-  * The first step that a client needs to do is to initialize and start a
-    YarnClient.
-
-+---+
-  YarnClient yarnClient = YarnClient.createYarnClient();
-  yarnClient.init(conf);
-  yarnClient.start();
-+---+
-
-  * Once a client is set up, the client needs to create an application, and get
-    its application id.
-
-+---+
-  YarnClientApplication app = yarnClient.createApplication();
-  GetNewApplicationResponse appResponse = app.getNewApplicationResponse();
-+---+
-
-  * The response from the <<<YarnClientApplication>>> for a new application also
-    contains information about the cluster such as the minimum/maximum resource
-    capabilities of the cluster. This is required so that to ensure that you can
-    correctly set the specifications of the container in which the
-    ApplicationMaster would be launched. Please refer to
-    <<<GetNewApplicationResponse>>> for more details.
-
-  * The main crux of a client is to setup the <<<ApplicationSubmissionContext>>>
-    which defines all the information needed by the RM to launch the AM. A client
-    needs to set the following into the context:
-
-    * Application info: id, name
-
-    * Queue, priority info: Queue to which the application will be submitted,
-      the priority to be assigned for the application.
-
-    * User: The user submitting the application
-
-    * <<<ContainerLaunchContext>>>: The information defining the container in
-      which the AM will be launched and run. The <<<ContainerLaunchContext>>>, as
-      mentioned previously, defines all the required information needed to run
-      the application such as the local <<R>>esources (binaries, jars, files
-      etc.), <<E>>nvironment settings (CLASSPATH etc.), the <<C>>ommand to be
-      executed and security <<T>>okens (<RECT>).
-
-    []
-
-+---+
-  // set the application submission context
-  ApplicationSubmissionContext appContext = app.getApplicationSubmissionContext();
-  ApplicationId appId = appContext.getApplicationId();
-
-  appContext.setKeepContainersAcrossApplicationAttempts(keepContainers);
-  appContext.setApplicationName(appName);
-
-  // set local resources for the application master
-  // local files or archives as needed
-  // In this scenario, the jar file for the application master is part of the local resources
-  Map<String, LocalResource> localResources = new HashMap<String, LocalResource>();
-
-  LOG.info("Copy App Master jar from local filesystem and add to local environment");
-  // Copy the application master jar to the filesystem
-  // Create a local resource to point to the destination jar path
-  FileSystem fs = FileSystem.get(conf);
-  addToLocalResources(fs, appMasterJar, appMasterJarPath, appId.toString(),
-      localResources, null);
-
-  // Set the log4j properties if needed
-  if (!log4jPropFile.isEmpty()) {
-    addToLocalResources(fs, log4jPropFile, log4jPath, appId.toString(),
-        localResources, null);
-  }
-
-  // The shell script has to be made available on the final container(s)
-  // where it will be executed.
-  // To do this, we need to first copy into the filesystem that is visible
-  // to the yarn framework.
-  // We do not need to set this as a local resource for the application
-  // master as the application master does not need it.
-  String hdfsShellScriptLocation = "";
-  long hdfsShellScriptLen = 0;
-  long hdfsShellScriptTimestamp = 0;
-  if (!shellScriptPath.isEmpty()) {
-    Path shellSrc = new Path(shellScriptPath);
-    String shellPathSuffix =
-        appName + "/" + appId.toString() + "/" + SCRIPT_PATH;
-    Path shellDst =
-        new Path(fs.getHomeDirectory(), shellPathSuffix);
-    fs.copyFromLocalFile(false, true, shellSrc, shellDst);
-    hdfsShellScriptLocation = shellDst.toUri().toString();
-    FileStatus shellFileStatus = fs.getFileStatus(shellDst);
-    hdfsShellScriptLen = shellFileStatus.getLen();
-    hdfsShellScriptTimestamp = shellFileStatus.getModificationTime();
-  }
-
-  if (!shellCommand.isEmpty()) {
-    addToLocalResources(fs, null, shellCommandPath, appId.toString(),
-        localResources, shellCommand);
-  }
-
-  if (shellArgs.length > 0) {
-    addToLocalResources(fs, null, shellArgsPath, appId.toString(),
-        localResources, StringUtils.join(shellArgs, " "));
-  }
-
-  // Set the env variables to be setup in the env where the application master will be run
-  LOG.info("Set the environment for the application master");
-  Map<String, String> env = new HashMap<String, String>();
-
-  // put location of shell script into env
-  // using the env info, the application master will create the correct local resource for the
-  // eventual containers that will be launched to execute the shell scripts
-  env.put(DSConstants.DISTRIBUTEDSHELLSCRIPTLOCATION, hdfsShellScriptLocation);
-  env.put(DSConstants.DISTRIBUTEDSHELLSCRIPTTIMESTAMP, Long.toString(hdfsShellScriptTimestamp));
-  env.put(DSConstants.DISTRIBUTEDSHELLSCRIPTLEN, Long.toString(hdfsShellScriptLen));
-
-  // Add AppMaster.jar location to classpath
-  // At some point we should not be required to add
-  // the hadoop specific classpaths to the env.
-  // It should be provided out of the box.
-  // For now setting all required classpaths including
-  // the classpath to "." for the application jar
-  StringBuilder classPathEnv = new StringBuilder(Environment.CLASSPATH.$$())
-    .append(ApplicationConstants.CLASS_PATH_SEPARATOR).append("./*");
-  for (String c : conf.getStrings(
-      YarnConfiguration.YARN_APPLICATION_CLASSPATH,
-      YarnConfiguration.DEFAULT_YARN_CROSS_PLATFORM_APPLICATION_CLASSPATH)) {
-    classPathEnv.append(ApplicationConstants.CLASS_PATH_SEPARATOR);
-    classPathEnv.append(c.trim());
-  }
-  classPathEnv.append(ApplicationConstants.CLASS_PATH_SEPARATOR).append(
-    "./log4j.properties");
-
-  // Set the necessary command to execute the application master
-  Vector<CharSequence> vargs = new Vector<CharSequence>(30);
-
-  // Set java executable command
-  LOG.info("Setting up app master command");
-  vargs.add(Environment.JAVA_HOME.$$() + "/bin/java");
-  // Set Xmx based on am memory size
-  vargs.add("-Xmx" + amMemory + "m");
-  // Set class name
-  vargs.add(appMasterMainClass);
-  // Set params for Application Master
-  vargs.add("--container_memory " + String.valueOf(containerMemory));
-  vargs.add("--container_vcores " + String.valueOf(containerVirtualCores));
-  vargs.add("--num_containers " + String.valueOf(numContainers));
-  vargs.add("--priority " + String.valueOf(shellCmdPriority));
-
-  for (Map.Entry<String, String> entry : shellEnv.entrySet()) {
-    vargs.add("--shell_env " + entry.getKey() + "=" + entry.getValue());
-  }
-  if (debugFlag) {
-    vargs.add("--debug");
-  }
-
-  vargs.add("1>" + ApplicationConstants.LOG_DIR_EXPANSION_VAR + "/AppMaster.stdout");
-  vargs.add("2>" + ApplicationConstants.LOG_DIR_EXPANSION_VAR + "/AppMaster.stderr");
-
-  // Get final commmand
-  StringBuilder command = new StringBuilder();
-  for (CharSequence str : vargs) {
-    command.append(str).append(" ");
-  }
-
-  LOG.info("Completed setting up app master command " + command.toString());
-  List<String> commands = new ArrayList<String>();
-  commands.add(command.toString());
-
-  // Set up the container launch context for the application master
-  ContainerLaunchContext amContainer = ContainerLaunchContext.newInstance(
-    localResources, env, commands, null, null, null);
-
-  // Set up resource type requirements
-  // For now, both memory and vcores are supported, so we set memory and
-  // vcores requirements
-  Resource capability = Resource.newInstance(amMemory, amVCores);
-  appContext.setResource(capability);
-
-  // Service data is a binary blob that can be passed to the application
-  // Not needed in this scenario
-  // amContainer.setServiceData(serviceData);
-
-  // Setup security tokens
-  if (UserGroupInformation.isSecurityEnabled()) {
-    // Note: Credentials class is marked as LimitedPrivate for HDFS and MapReduce
-    Credentials credentials = new Credentials();
-    String tokenRenewer = conf.get(YarnConfiguration.RM_PRINCIPAL);
-    if (tokenRenewer == null || tokenRenewer.length() == 0) {
-      throw new IOException(
-        "Can't get Master Kerberos principal for the RM to use as renewer");
-    }
-
-    // For now, only getting tokens for the default file-system.
-    final Token<?> tokens[] =
-        fs.addDelegationTokens(tokenRenewer, credentials);
-    if (tokens != null) {
-      for (Token<?> token : tokens) {
-        LOG.info("Got dt for " + fs.getUri() + "; " + token);
-      }
-    }
-    DataOutputBuffer dob = new DataOutputBuffer();
-    credentials.writeTokenStorageToStream(dob);
-    ByteBuffer fsTokens = ByteBuffer.wrap(dob.getData(), 0, dob.getLength());
-    amContainer.setTokens(fsTokens);
-  }
-
-  appContext.setAMContainerSpec(amContainer);
-+---+
-
-  * After the setup process is complete, the client is ready to submit
-    the application with specified priority and queue.
-
-+---+
-  // Set the priority for the application master
-  Priority pri = Priority.newInstance(amPriority);
-  appContext.setPriority(pri);
-
-  // Set the queue to which this application is to be submitted in the RM
-  appContext.setQueue(amQueue);
-
-  // Submit the application to the applications manager
-  // SubmitApplicationResponse submitResp = applicationsManager.submitApplication(appRequest);
-
-  yarnClient.submitApplication(appContext);
-+---+
-
-  * At this point, the RM will have accepted the application and in the
-    background, will go through the process of allocating a container with the
-    required specifications and then eventually setting up and launching the AM
-    on the allocated container.
-
-  * There are multiple ways a client can track progress of the actual task.
-
-    * It can communicate with the RM and request for a report of the application
-      via the <<<getApplicationReport()>>> method of <<<YarnClient>>>.
-
-+-----+
-  // Get application report for the appId we are interested in
-  ApplicationReport report = yarnClient.getApplicationReport(appId);
-+-----+
-
-      The <<<ApplicationReport>>> received from the RM consists of the following:
-
-        * General application information: Application id, queue to which the
-          application was submitted, user who submitted the application and the
-          start time for the application.
-
-        * ApplicationMaster details: the host on which the AM is running, the
-          rpc port (if any) on which it is listening for requests from clients
-          and a token that the client needs to communicate with the AM.
-
-        * Application tracking information: If the application supports some form
-          of progress tracking, it can set a tracking url which is available via
-          <<<ApplicationReport>>>'s <<<getTrackingUrl()>>> method that a client
-          can look at to monitor progress.
-
-        * Application status: The state of the application as seen by the
-          ResourceManager is available via
-          <<<ApplicationReport#getYarnApplicationState>>>. If the
-          <<<YarnApplicationState>>> is set to <<<FINISHED>>>, the client should
-          refer to <<<ApplicationReport#getFinalApplicationStatus>>> to check for
-          the actual success/failure of the application task itself. In case of
-          failures, <<<ApplicationReport#getDiagnostics>>> may be useful to shed
-          some more light on the the failure.
-
-    * If the ApplicationMaster supports it, a client can directly query the AM
-      itself for progress updates via the host:rpcport information obtained from
-      the application report. It can also use the tracking url obtained from the
-      report if available.
-
-  * In certain situations, if the application is taking too long or due to other
-    factors, the client may wish to kill the application. <<<YarnClient>>>
-    supports the <<<killApplication>>> call that allows a client to send a kill
-    signal to the AM via the ResourceManager. An ApplicationMaster if so
-    designed may also support an abort call via its rpc layer that a client may
-    be able to leverage.
-
-+---+
-  yarnClient.killApplication(appId);
-+---+
-
-** Writing an ApplicationMaster (AM)
-
-  * The AM is the actual owner of the job. It will be launched
-    by the RM and via the client will be provided all the
-    necessary information and resources about the job that it has been tasked
-    with to oversee and complete.
-
-  * As the AM is launched within a container that may (likely
-    will) be sharing a physical host with other containers, given the
-    multi-tenancy nature, amongst other issues, it cannot make any assumptions
-    of things like pre-configured ports that it can listen on.
-
-  * When the AM starts up, several parameters are made available
-    to it via the environment. These include the <<<ContainerId>>> for the
-    AM container, the application submission time and details
-    about the NM (NodeManager) host running the ApplicationMaster.
-    Ref <<<ApplicationConstants>>> for parameter names.
-
-  * All interactions with the RM require an <<<ApplicationAttemptId>>> (there can
-    be multiple attempts per application in case of failures). The
-    <<<ApplicationAttemptId>>> can be obtained from the AM's container id. There
-    are helper APIs to convert the value obtained from the environment into
-    objects.
-
-+---+
-  Map<String, String> envs = System.getenv();
-  String containerIdString =
-      envs.get(ApplicationConstants.AM_CONTAINER_ID_ENV);
-  if (containerIdString == null) {
-    // container id should always be set in the env by the framework
-    throw new IllegalArgumentException(
-        "ContainerId not set in the environment");
-  }
-  ContainerId containerId = ConverterUtils.toContainerId(containerIdString);
-  ApplicationAttemptId appAttemptID = containerId.getApplicationAttemptId();
-+---+
-
-  * After an AM has initialized itself completely, we can start the two clients:
-    one to ResourceManager, and one to NodeManagers. We set them up with our
-    customized event handler, and we will talk about those event handlers in
-    detail later in this article.
-
-+---+
-  AMRMClientAsync.CallbackHandler allocListener = new RMCallbackHandler();
-  amRMClient = AMRMClientAsync.createAMRMClientAsync(1000, allocListener);
-  amRMClient.init(conf);
-  amRMClient.start();
-
-  containerListener = createNMCallbackHandler();
-  nmClientAsync = new NMClientAsyncImpl(containerListener);
-  nmClientAsync.init(conf);
-  nmClientAsync.start();
-+---+
-
-  * The AM has to emit heartbeats to the RM to keep it informed that the AM is
-    alive and still running. The timeout expiry interval at the RM is defined by
-    a config setting accessible via
-    <<<YarnConfiguration.RM_AM_EXPIRY_INTERVAL_MS>>> with the default being
-    defined by <<<YarnConfiguration.DEFAULT_RM_AM_EXPIRY_INTERVAL_MS>>>. The
-    ApplicationMaster needs to register itself with the ResourceManager to
-    start hearbeating.
-
-+---+
-  // Register self with ResourceManager
-  // This will start heartbeating to the RM
-  appMasterHostname = NetUtils.getHostname();
-  RegisterApplicationMasterResponse response = amRMClient
-      .registerApplicationMaster(appMasterHostname, appMasterRpcPort,
-          appMasterTrackingUrl);
-+---+
-
-  * In the response of the registration, maximum resource capability if included. You may want to use this to check the application's request.
-
-+---+
-  // Dump out information about cluster capability as seen by the
-  // resource manager
-  int maxMem = response.getMaximumResourceCapability().getMemory();
-  LOG.info("Max mem capabililty of resources in this cluster " + maxMem);
-
-  int maxVCores = response.getMaximumResourceCapability().getVirtualCores();
-  LOG.info("Max vcores capabililty of resources in this cluster " + maxVCores);
-
-  // A resource ask cannot exceed the max.
-  if (containerMemory > maxMem) {
-    LOG.info("Container memory specified above max threshold of cluster."
-        + " Using max value." + ", specified=" + containerMemory + ", max="
-        + maxMem);
-    containerMemory = maxMem;
-  }
-
-  if (containerVirtualCores > maxVCores) {
-    LOG.info("Container virtual cores specified above max threshold of  cluster."
-      + " Using max value." + ", specified=" + containerVirtualCores + ", max="
-      + maxVCores);
-    containerVirtualCores = maxVCores;
-  }
-  List<Container> previousAMRunningContainers =
-      response.getContainersFromPreviousAttempts();
-  LOG.info("Received " + previousAMRunningContainers.size()
-          + " previous AM's running containers on AM registration.");
-+---+
-
-  * Based on the task requirements, the AM can ask for a set of containers to run
-    its tasks on. We can now calculate how many containers we need, and request
-    those many containers.
-
-+---+
-  List<Container> previousAMRunningContainers =
-      response.getContainersFromPreviousAttempts();
-  List<Container> previousAMRunningContainers =
-      response.getContainersFromPreviousAttempts();
-  LOG.info("Received " + previousAMRunningContainers.size()
-      + " previous AM's running containers on AM registration.");
-
-  int numTotalContainersToRequest =
-      numTotalContainers - previousAMRunningContainers.size();
-  // Setup ask for containers from RM
-  // Send request for containers to RM
-  // Until we get our fully allocated quota, we keep on polling RM for
-  // containers
-  // Keep looping until all the containers are launched and shell script
-  // executed on them ( regardless of success/failure).
-  for (int i = 0; i < numTotalContainersToRequest; ++i) {
-    ContainerRequest containerAsk = setupContainerAskForRM();
-    amRMClient.addContainerRequest(containerAsk);
-  }
-+---+
-
-  * In <<<setupContainerAskForRM()>>>, the follow two things need some set up:
-
-    * Resource capability: Currently, YARN supports memory based resource
-      requirements so the request should define how much memory is needed. The
-      value is defined in MB and has to less than the max capability of the
-      cluster and an exact multiple of the min capability. Memory resources
-      correspond to physical memory limits imposed on the task containers. It
-      will also support computation based resource (vCore), as shown in the code.
-
-    * Priority: When asking for sets of containers, an AM may define different
-      priorities to each set. For example, the Map-Reduce AM may assign a higher
-      priority to containers needed for the Map tasks and a lower priority for
-      the Reduce tasks' containers.
-
-    []
-
-+---+
-  private ContainerRequest setupContainerAskForRM() {
-    // setup requirements for hosts
-    // using * as any host will do for the distributed shell app
-    // set the priority for the request
-    Priority pri = Priority.newInstance(requestPriority);
-
-    // Set up resource type requirements
-    // For now, memory and CPU are supported so we set memory and cpu requirements
-    Resource capability = Resource.newInstance(containerMemory,
-      containerVirtualCores);
-
-    ContainerRequest request = new ContainerRequest(capability, null, null,
-        pri);
-    LOG.info("Requested container ask: " + request.toString());
-    return request;
-  }
-+---+
-
-  * After container allocation requests have been sent by the application
-    manager, contailers will be launched asynchronously, by the event handler of
-    the <<<AMRMClientAsync>>> client. The handler should implement
-    <<<AMRMClientAsync.CallbackHandler>>> interface.
-
-    * When there are containers allocated, the handler sets up a thread that runs
-      the code to launch containers. Here we use the name
-      <<<LaunchContainerRunnable>>> to demonstrate. We will talk about the
-      <<<LaunchContainerRunnable>>> class in the following part of this article.
-
-+---+
-  @Override
-  public void onContainersAllocated(List<Container> allocatedContainers) {
-    LOG.info("Got response from RM for container ask, allocatedCnt="
-        + allocatedContainers.size());
-    numAllocatedContainers.addAndGet(allocatedContainers.size());
-    for (Container allocatedContainer : allocatedContainers) {
-      LaunchContainerRunnable runnableLaunchContainer =
-          new LaunchContainerRunnable(allocatedContainer, containerListener);
-      Thread launchThread = new Thread(runnableLaunchContainer);
-
-      // launch and start the container on a separate thread to keep
-      // the main thread unblocked
-      // as all containers may not be allocated at one go.
-      launchThreads.add(launchThread);
-      launchThread.start();
-    }
-  }
-+---+
-
-    * On heart beat, the event handler reports the progress of the application.
-
-+---+
-  @Override
-  public float getProgress() {
-    // set progress to deliver to RM on next heartbeat
-    float progress = (float) numCompletedContainers.get()
-        / numTotalContainers;
-    return progress;
-  }
-+---+
-
-    []
-
-  * The container launch thread actually launches the containers on NMs. After a
-    container has been allocated to the AM, it needs to follow a similar process
-    that the client followed in setting up the <<<ContainerLaunchContext>>> for
-    the eventual task that is going to be running on the allocated Container.
-    Once the <<<ContainerLaunchContext>>> is defined, the AM can start it through
-    the <<<NMClientAsync>>>.
-
-+---+
-  // Set the necessary command to execute on the allocated container
-  Vector<CharSequence> vargs = new Vector<CharSequence>(5);
-
-  // Set executable command
-  vargs.add(shellCommand);
-  // Set shell script path
-  if (!scriptPath.isEmpty()) {
-    vargs.add(Shell.WINDOWS ? ExecBatScripStringtPath
-      : ExecShellStringPath);
-  }
-
-  // Set args for the shell command if any
-  vargs.add(shellArgs);
-  // Add log redirect params
-  vargs.add("1>" + ApplicationConstants.LOG_DIR_EXPANSION_VAR + "/stdout");
-  vargs.add("2>" + ApplicationConstants.LOG_DIR_EXPANSION_VAR + "/stderr");
-
-  // Get final commmand
-  StringBuilder command = new StringBuilder();
-  for (CharSequence str : vargs) {
-    command.append(str).append(" ");
-  }
-
-  List<String> commands = new ArrayList<String>();
-  commands.add(command.toString());
-
-  // Set up ContainerLaunchContext, setting local resource, environment,
-  // command and token for constructor.
-
-  // Note for tokens: Set up tokens for the container too. Today, for normal
-  // shell commands, the container in distribute-shell doesn't need any
-  // tokens. We are populating them mainly for NodeManagers to be able to
-  // download anyfiles in the distributed file-system. The tokens are
-  // otherwise also useful in cases, for e.g., when one is running a
-  // "hadoop dfs" command inside the distributed shell.
-  ContainerLaunchContext ctx = ContainerLaunchContext.newInstance(
-    localResources, shellEnv, commands, null, allTokens.duplicate(), null);
-  containerListener.addContainer(container.getId(), container);
-  nmClientAsync.startContainerAsync(container, ctx);
-+---+
-
-  * The <<<NMClientAsync>>> object, together with its event handler, handles container events. Including container start, stop, status update, and occurs an error.
-  
-  * After the ApplicationMaster determines the work is done, it needs to unregister itself through the AM-RM client, and then stops the client. 
-
-+---+
-  try {
-    amRMClient.unregisterApplicationMaster(appStatus, appMessage, null);
-  } catch (YarnException ex) {
-    LOG.error("Failed to unregister application", ex);
-  } catch (IOException e) {
-    LOG.error("Failed to unregister application", e);
-  }
-  
-  amRMClient.stop();
-+---+
-
-~~** Defining the context in which your code runs
-
-~~*** Container Resource Requests
-
-~~*** Local Resources
-
-~~*** Environment
-
-~~**** Managing the CLASSPATH
-
-~~** Security
-
-* FAQ
-
-** How can I distribute my application's jars to all of the nodes in the YARN
-   cluster that need it?
-
-  * You can use the LocalResource to add resources to your application request.
-    This will cause YARN to distribute the resource to the ApplicationMaster
-    node. If the resource is a tgz, zip, or jar - you can have YARN unzip it.
-    Then, all you need to do is add the unzipped folder to your classpath. For
-    example, when creating your application request:
-
-+---+
-  File packageFile = new File(packagePath);
-  Url packageUrl = ConverterUtils.getYarnUrlFromPath(
-      FileContext.getFileContext.makeQualified(new Path(packagePath)));
-
-  packageResource.setResource(packageUrl);
-  packageResource.setSize(packageFile.length());
-  packageResource.setTimestamp(packageFile.lastModified());
-  packageResource.setType(LocalResourceType.ARCHIVE);
-  packageResource.setVisibility(LocalResourceVisibility.APPLICATION);
-
-  resource.setMemory(memory);
-  containerCtx.setResource(resource);
-  containerCtx.setCommands(ImmutableList.of(
-      "java -cp './package/*' some.class.to.Run "
-      + "1>" + ApplicationConstants.LOG_DIR_EXPANSION_VAR + "/stdout "
-      + "2>" + ApplicationConstants.LOG_DIR_EXPANSION_VAR + "/stderr"));
-  containerCtx.setLocalResources(
-      Collections.singletonMap("package", packageResource));
-  appCtx.setApplicationId(appId);
-  appCtx.setUser(user.getShortUserName);
-  appCtx.setAMContainerSpec(containerCtx);
-  yarnClient.submitApplication(appCtx);
-+---+
-
-  As you can see, the <<<setLocalResources>>> command takes a map of names to
-  resources. The name becomes a sym link in your application's cwd, so you can
-  just refer to the artifacts inside by using ./package/*.
-
-  Note: Java's classpath (cp) argument is VERY sensitive.
-  Make sure you get the syntax EXACTLY correct.
-
-  Once your package is distributed to your AM, you'll need to follow the same
-  process whenever your AM starts a new container (assuming you want the
-  resources to be sent to your container). The code for this is the same. You
-  just need to make sure that you give your AM the package path (either HDFS, or
-  local), so that it can send the resource URL along with the container ctx.
-
-** How do I get the ApplicationMaster's <<<ApplicationAttemptId>>>?
-
-  * The <<<ApplicationAttemptId>>> will be passed to the AM via the environment
-    and the value from the environment can be converted into an
-    <<<ApplicationAttemptId>>> object via the ConverterUtils helper function.
-
-** Why my container is killed by the NodeManager?
-
-  * This is likely due to high memory usage exceeding your requested container
-    memory size. There are a number of reasons that can cause this. First, look
-    at the process tree that the NodeManager dumps when it kills your container.
-    The two things you're interested in are physical memory and virtual memory.
-    If you have exceeded physical memory limits your app is using too much
-    physical memory. If you're running a Java app, you can use -hprof to look at
-    what is taking up space in the heap. If you have exceeded virtual memory, you
-    may need to increase the value of the the cluster-wide configuration variable
-    <<<yarn.nodemanager.vmem-pmem-ratio>>>.
-
-** How do I include native libraries?
-
-  * Setting <<<-Djava.library.path>>> on the command line while launching a
-    container can cause native libraries used by Hadoop to not be loaded
-    correctly and can result in errors. It is cleaner to use
-    <<<LD_LIBRARY_PATH>>> instead.
-
-* Useful Links
-
-  * {{{http://hadoop.apache.org/docs/current/hadoop-yarn/hadoop-yarn-site/YARN.html}YARN Architecture}}
-
-  * {{{http://hadoop.apache.org/docs/current/hadoop-yarn/hadoop-yarn-site/CapacityScheduler.html}YARN Capacity Scheduler}}
-
-  * {{{http://hadoop.apache.org/docs/current/hadoop-yarn/hadoop-yarn-site/FairScheduler.html}YARN Fair Scheduler}}
-
-* Sample code
-
-  * Yarn distributed shell: in <<<hadoop-yarn-applications-distributedshell>>>
-    project after you set up your development environment.
-

http://git-wip-us.apache.org/repos/asf/hadoop/blob/06aca7c6/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/apt/YARN.apt.vm
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/apt/YARN.apt.vm b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/apt/YARN.apt.vm
deleted file mode 100644
index 465c5d1..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/apt/YARN.apt.vm
+++ /dev/null
@@ -1,77 +0,0 @@
-~~ 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.
-
-  ---
-  YARN
-  ---
-  ---
-  ${maven.build.timestamp}
-
-Apache Hadoop NextGen MapReduce (YARN)
-
-  MapReduce has undergone a complete overhaul in hadoop-0.23 and we now have, 
-  what we call, MapReduce 2.0 (MRv2) or YARN.
-
-  The fundamental idea of MRv2 is to split up the two major functionalities of 
-  the JobTracker, resource management and job scheduling/monitoring, into 
-  separate daemons. The idea is to have a global ResourceManager (<RM>) and 
-  per-application ApplicationMaster (<AM>).  An application is either a single 
-  job in the classical sense of Map-Reduce jobs or a DAG of jobs. 
-
-  The ResourceManager and per-node slave, the NodeManager (<NM>), form the 
-  data-computation framework. The ResourceManager is the ultimate authority that 
-  arbitrates resources among all the applications in the system. 
-
-  The per-application ApplicationMaster is, in effect, a framework specific 
-  library and is tasked with negotiating resources from the ResourceManager and 
-  working with the NodeManager(s) to execute and monitor the tasks.
-
-[./yarn_architecture.gif] MapReduce NextGen Architecture
-
-  The ResourceManager has two main components: Scheduler and 
-  ApplicationsManager.
-
-  The Scheduler is responsible for allocating resources to the various running 
-  applications subject to familiar constraints of capacities, queues etc. The 
-  Scheduler is pure scheduler in the sense that it performs no monitoring or 
-  tracking of status for the application. Also, it offers no guarantees about 
-  restarting failed tasks either due to application failure or hardware 
-  failures. The Scheduler performs its scheduling function based the resource 
-  requirements of the applications; it does so based on the abstract notion of 
-  a resource <Container> which incorporates elements such as memory, cpu, disk, 
-  network etc. In the first version, only <<<memory>>> is supported. 
-
-  The Scheduler has a pluggable policy plug-in, which is responsible for 
-  partitioning the cluster resources among the various queues, applications etc. 
-  The current Map-Reduce schedulers such as the CapacityScheduler and the 
-  FairScheduler would be some examples of the plug-in. 
-
-  The CapacityScheduler supports <<<hierarchical queues>>> to allow for more 
-  predictable sharing of cluster resources
-
-  The ApplicationsManager is responsible for accepting job-submissions, 
-  negotiating the first container for executing the application specific 
-  ApplicationMaster and provides the service for restarting the
-  ApplicationMaster container on failure.
-
-  The NodeManager is the per-machine framework agent who is responsible for 
-  containers, monitoring their resource usage (cpu, memory, disk, network) and 
-  reporting the same to the ResourceManager/Scheduler.
-
-  The per-application ApplicationMaster has the responsibility of negotiating 
-  appropriate resource containers from the Scheduler, tracking their status and 
-  monitoring for progress.
-
-  MRV2 maintains <<API compatibility>> with previous stable release 
-  (hadoop-1.x).  This means that all Map-Reduce jobs should still run 
-  unchanged on top of MRv2 with just a recompile.
-

http://git-wip-us.apache.org/repos/asf/hadoop/blob/06aca7c6/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/apt/YarnCommands.apt.vm
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/apt/YarnCommands.apt.vm b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/apt/YarnCommands.apt.vm
deleted file mode 100644
index 67f8a58..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/apt/YarnCommands.apt.vm
+++ /dev/null
@@ -1,369 +0,0 @@
-~~ 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.
-
-  ---
-  YARN Commands
-  ---
-  ---
-  ${maven.build.timestamp}
-
-YARN Commands
-
-%{toc|section=1|fromDepth=0}
-
-* Overview
-
-  YARN commands are invoked by the bin/yarn script. Running the yarn script
-  without any arguments prints the description for all commands.
-
- Usage: <<<yarn [SHELL_OPTIONS] COMMAND [GENERIC_OPTIONS] [COMMAND_OPTIONS]>>>
-
-  YARN has an option parsing framework that employs parsing generic options as
-  well as running classes.
-
-*---------------+--------------+
-|| COMMAND_OPTIONS || Description                   |
-*-------------------------+-------------+
-| SHELL_OPTIONS | The common set of shell options. These are documented on the {{{../../hadoop-project-dist/hadoop-common/CommandsManual.html#Shell Options}Commands Manual}} page.
-*-------------------------+----+
-| GENERIC_OPTIONS | The common set of options supported by multiple commands. See the Hadoop {{{../../hadoop-project-dist/hadoop-common/CommandsManual.html#Generic Options}Commands Manual}} for more information.
-*------------------+---------------+
-| COMMAND COMMAND_OPTIONS | Various commands with their options are described
-|                         | in the following sections. The commands have been
-|                         | grouped into {{User Commands}} and
-|                         | {{Administration Commands}}.
-*-------------------------+--------------+
-
-* {User Commands}
-
-  Commands useful for users of a Hadoop cluster.
-
-** <<<application>>>
-
-  Usage: <<<yarn application [options] >>>
-
-*---------------+--------------+
-|| COMMAND_OPTIONS || Description                   |
-*---------------+--------------+
-| -appStates States | Works with -list to filter applications based on input
-|                   | comma-separated list of application states. The valid
-|                   | application state can be one of the following: \
-|                   | ALL, NEW, NEW_SAVING, SUBMITTED, ACCEPTED, RUNNING,
-|                   | FINISHED, FAILED, KILLED
-*---------------+--------------+
-| -appTypes Types | Works with -list to filter applications based on input
-|                 | comma-separated list of application types.
-*---------------+--------------+
-| -list | Lists applications from the RM. Supports optional use of -appTypes
-|       | to filter applications based on application type, and -appStates to
-|       | filter applications based on application state.
-*---------------+--------------+
-| -kill ApplicationId | Kills the application.
-*---------------+--------------+
-| -status  ApplicationId | Prints the status of the application.
-*---------------+--------------+
-
-  Prints application(s) report/kill application
-
-** <<<applicationattempt>>>
-
-  Usage: <<<yarn applicationattempt [options] >>>
-
-*---------------+--------------+
-|| COMMAND_OPTIONS || Description                   |
-*---------------+--------------+
-| -help | Help
-*---------------+--------------+
-| -list ApplicationId | Lists applications attempts from the RM
-*---------------+--------------+
-| -status  Application Attempt Id | Prints the status of the application attempt.
-*---------------+--------------+
-
-  prints applicationattempt(s) report
-
-** <<<classpath>>>
-
-  Usage: <<<yarn classpath>>>
-
-  Prints the class path needed to get the Hadoop jar and the required libraries
-
-
-** <<<container>>>
-
-  Usage: <<<yarn container [options] >>>
-
-*---------------+--------------+
-|| COMMAND_OPTIONS || Description                   |
-*---------------+--------------+
-| -help | Help
-*---------------+--------------+
-| -list ApplicationId | Lists containers for the application attempt.
-*---------------+--------------+
-| -status  ContainerId | Prints the status of the container.
-*---------------+--------------+
-
-  prints container(s) report
-
-** <<<jar>>>
-
-  Usage: <<<yarn jar <jar> [mainClass] args... >>>
-
-  Runs a jar file. Users can bundle their YARN code in a jar file and execute
-  it using this command.
-
-** <<<logs>>>
-
-  Usage: <<<yarn logs -applicationId <application ID> [options] >>>
-
-*---------------+--------------+
-|| COMMAND_OPTIONS || Description                   |
-*---------------+--------------+
-| -applicationId \<application ID\> | Specifies an application id |
-*---------------+--------------+
-| -appOwner AppOwner | AppOwner (assumed to be current user if not
-|                    | specified)
-*---------------+--------------+
-| -containerId ContainerId | ContainerId (must be specified if node address is
-|                          | specified)
-*---------------+--------------+
-| -help | Help
-*---------------+--------------+
-| -nodeAddress NodeAddress | NodeAddress in the format nodename:port (must be
-|                          | specified if container id is specified)
-*---------------+--------------+
-
-  Dump the container logs
-
-
-** <<<node>>>
-
-  Usage: <<<yarn node [options] >>>
-
-*---------------+--------------+
-|| COMMAND_OPTIONS || Description                   |
-*---------------+--------------+
-| -all | Works with -list to list all nodes.
-*---------------+--------------+
-| -list | Lists all running nodes. Supports optional use of -states to filter
-|       | nodes based on node state, and -all to list all nodes.
-*---------------+--------------+
-| -states States | Works with -list to filter nodes based on input
-|                | comma-separated list of node states.
-*---------------+--------------+
-| -status NodeId | Prints the status report of the node.
-*---------------+--------------+
-
-  Prints node report(s)
-
-** <<<queue>>>
-
-  Usage: <<<yarn queue [options] >>>
-
-*---------------+--------------+
-|| COMMAND_OPTIONS || Description                   |
-*---------------+--------------+
-| -help | Help
-*---------------+--------------+
-| -status  QueueName | Prints the status of the queue.
-*---------------+--------------+
-
-  Prints queue information
-
-
-** <<<version>>>
-
-  Usage: <<<yarn version>>>
-
-  Prints the Hadoop version.
-
-* {Administration Commands}
-
-  Commands useful for administrators of a Hadoop cluster.
-
-** <<<daemonlog>>>
-
-  Usage:
-
----------------------------------
-   yarn daemonlog -getlevel <host:httpport> <classname>
-   yarn daemonlog -setlevel <host:httpport> <classname> <level>
----------------------------------
-
-*---------------+--------------+
-|| COMMAND_OPTIONS || Description                   |
-*---------------+--------------+
-| -getlevel \<host:httpport\> \<classname\> | Prints the log level of the log identified  
-| | by a qualified \<classname\>, in the daemon running at \<host:httpport\>. This 
-| | command internally connects to http://\<host:httpport\>/logLevel?log=\<classname\>
-*---------------+--------------+
-| -setlevel \<host:httpport\> \<classname\> \<level\> | Sets the log level of the log 
-| | identified by a qualified \<classname\> in the daemon running at \<host:httpport\>. 
-| | This command internally connects to http://\<host:httpport\>/logLevel?log=\<classname\>&level=\<level\>
-*---------------+--------------+
-
-  Get/Set the log level for a Log identified by a qualified class name in the daemon.
-
-----
-  Example: $ bin/yarn daemonlog -setlevel 127.0.0.1:8088 org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppImpl DEBUG
-----
-
-** <<<nodemanager>>>
-
-  Usage: <<<yarn nodemanager>>>
-
-  Start the NodeManager
-
-** <<<proxyserver>>>
-
-  Usage: <<<yarn proxyserver>>>
-
-  Start the web proxy server
-
-** <<<resourcemanager>>>
-
-  Usage: <<<yarn resourcemanager [-format-state-store]>>>
-
-*---------------+--------------+
-|| COMMAND_OPTIONS || Description                   |
-*---------------+--------------+
-| -format-state-store | Formats the RMStateStore. This will clear the
-|                     | RMStateStore and is useful if past applications are no
-|                     | longer needed. This should be run only when the
-|                     | ResourceManager is not running.
-*---------------+--------------+
-
-  Start the ResourceManager
-
-
-** <<<rmadmin>>>
-
-  Usage:
-
-----
-  yarn rmadmin [-refreshQueues]
-               [-refreshNodes]
-               [-refreshUserToGroupsMapping] 
-               [-refreshSuperUserGroupsConfiguration]
-               [-refreshAdminAcls] 
-               [-refreshServiceAcl]
-               [-getGroups [username]]
-               [-transitionToActive [--forceactive] [--forcemanual] <serviceId>]
-               [-transitionToStandby [--forcemanual] <serviceId>]
-               [-failover [--forcefence] [--forceactive] <serviceId1> <serviceId2>]
-               [-getServiceState <serviceId>]
-               [-checkHealth <serviceId>]
-               [-help [cmd]]
-----
-
-*---------------+--------------+
-|| COMMAND_OPTIONS || Description                   |
-*---------------+--------------+
-| -refreshQueues | Reload the queues' acls, states and scheduler specific
-|                | properties. ResourceManager will reload the mapred-queues
-|                | configuration file.
-*---------------+--------------+
-| -refreshNodes | Refresh the hosts information at the ResourceManager. |
-*---------------+--------------+
-| -refreshUserToGroupsMappings| Refresh user-to-groups mappings. |
-*---------------+--------------+
-| -refreshSuperUserGroupsConfiguration | Refresh superuser proxy groups
-|                                      | mappings.
-*---------------+--------------+
-| -refreshAdminAcls | Refresh acls for administration of ResourceManager |
-*---------------+--------------+
-| -refreshServiceAcl | Reload the service-level authorization policy file
-|                    | ResourceManager will reload the authorization policy
-|                    | file.
-*---------------+--------------+
-| -getGroups [username] | Get groups the specified user belongs to.
-*---------------+--------------+
-| -transitionToActive [--forceactive] [--forcemanual] \<serviceId\> |
-|               | Transitions the service into Active state.
-|               | Try to make the target active
-|               | without checking that there is no active node
-|               | if the --forceactive option is used.
-|               | This command can not be used if automatic failover is enabled.
-|               | Though you can override this by --forcemanual option,
-|               | you need caution.
-*---------------+--------------+
-| -transitionToStandby [--forcemanual] \<serviceId\> |
-|               | Transitions the service into Standby state.
-|               | This command can not be used if automatic failover is enabled.
-|               | Though you can override this by --forcemanual option,
-|               | you need caution.
-*---------------+--------------+
-| -failover [--forceactive] \<serviceId1\> \<serviceId2\> |
-|               | Initiate a failover from serviceId1 to serviceId2.
-|               | Try to failover to the target service even if it is not ready
-|               | if the --forceactive option is used.
-|               | This command can not be used if automatic failover is enabled.
-*---------------+--------------+
-| -getServiceState \<serviceId\> | Returns the state of the service.
-*---------------+--------------+
-| -checkHealth \<serviceId\> | Requests that the service perform a health
-|                            | check. The RMAdmin tool will exit with a
-|                            | non-zero exit code if the check fails.
-*---------------+--------------+
-| -help [cmd] | Displays help for the given command or all commands if none is
-|             | specified.
-*---------------+--------------+
-
-
-  Runs ResourceManager admin client
-
-** scmadmin
-
-  Usage: <<<yarn scmadmin [options] >>>
-
-*---------------+--------------+
-|| COMMAND_OPTIONS || Description                   |
-*---------------+--------------+
-| -help | Help
-*---------------+--------------+
-| -runCleanerTask | Runs the cleaner task
-*---------------+--------------+
-
-  Runs Shared Cache Manager admin client
-
-
-** sharedcachemanager
-
-  Usage: <<<yarn sharedcachemanager>>>
-
-  Start the Shared Cache Manager
-
-** timelineserver
-
-  Usage: <<<yarn timelineserver>>>
-
-  Start the TimeLineServer
-
-
-* Files
-
-** <<etc/hadoop/hadoop-env.sh>>
-
-    This file stores the global settings used by all Hadoop shell commands.
-
-** <<etc/hadoop/yarn-env.sh>>
-
-    This file stores overrides used by all YARN shell commands.
-
-** <<etc/hadoop/hadoop-user-functions.sh>>
-
-    This file allows for advanced users to override some shell functionality.
-
-** <<~/.hadooprc>>
-
-    This stores the personal environment for an individual user.  It is
-    processed after the <<<hadoop-env.sh>>>, <<<hadoop-user-functions.sh>>>, and <<<yarn-env.sh>>> files
-    and can contain the same settings.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/06aca7c6/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/apt/index.apt.vm
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/apt/index.apt.vm b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/apt/index.apt.vm
deleted file mode 100644
index 43e5b02..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/apt/index.apt.vm
+++ /dev/null
@@ -1,82 +0,0 @@
-~~ 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.
-
-  ---
-  Apache Hadoop NextGen MapReduce
-  ---
-  ---
-  ${maven.build.timestamp}
-  
-MapReduce NextGen aka YARN aka MRv2
-
-  The new architecture introduced in hadoop-0.23, divides the two major 
-  functions of the JobTracker: resource management and job life-cycle management 
-  into separate components.
-
-  The new ResourceManager manages the global assignment of compute resources to 
-  applications and the per-application ApplicationMaster manages the 
-  application’s scheduling and coordination. 
-  
-  An application is either a single job in the sense of classic MapReduce jobs 
-  or a DAG of such jobs. 
-  
-  The ResourceManager and per-machine NodeManager daemon, which manages the 
-  user processes on that machine, form the computation fabric. 
-  
-  The per-application ApplicationMaster is, in effect, a framework specific 
-  library and is tasked with negotiating resources from the ResourceManager and 
-  working with the NodeManager(s) to execute and monitor the tasks.
-
-  More details are available in the {{{./YARN.html}Architecture}} document.
-
-
-Documentation Index
-
-* YARN
-
-  * {{{./YARN.html}YARN Architecture}}
- 
-  * {{{./CapacityScheduler.html}Capacity Scheduler}}
- 
-  * {{{./FairScheduler.html}Fair Scheduler}}
- 
-  * {{{./ResourceManagerRestart.htaml}ResourceManager Restart}}
- 
-  * {{{./ResourceManagerHA.html}ResourceManager HA}}
- 
-  * {{{./WebApplicationProxy.html}Web Application Proxy}}
- 
-  * {{{./TimelineServer.html}YARN Timeline Server}}
- 
-  * {{{./WritingYarnApplications.html}Writing YARN Applications}}
- 
-  * {{{./YarnCommands.html}YARN Commands}}
- 
-  * {{{hadoop-sls/SchedulerLoadSimulator.html}Scheduler Load Simulator}}
- 
-  * {{{./NodeManagerRestart.html}NodeManager Restart}}
- 
-  * {{{./DockerContainerExecutor.html}DockerContainerExecutor}}
- 
-  * {{{./NodeManagerCGroups.html}Using CGroups}}
- 
-  * {{{./SecureContainer.html}Secure Containers}}
- 
-  * {{{./registry/index.html}Registry}}
-
-* YARN REST APIs
-
-  * {{{./WebServicesIntro.html}Introduction}}
-
-  * {{{./ResourceManagerRest.html}Resource Manager}}
-
-  * {{{./NodeManagerRest.html}Node Manager}}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/06aca7c6/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/CapacityScheduler.md
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/CapacityScheduler.md b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/CapacityScheduler.md
new file mode 100644
index 0000000..3c32cdd
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/CapacityScheduler.md
@@ -0,0 +1,186 @@
+<!---
+  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: Capacity Scheduler
+==========================
+
+* [Purpose](#Purpose)
+* [Overview](#Overview)
+* [Features](#Features)
+* [Configuration](#Configuration)
+    * [Setting up `ResourceManager` to use `CapacityScheduler`](#Setting_up_ResourceManager_to_use_CapacityScheduler`)
+    * [Setting up queues](#Setting_up_queues)
+    * [Queue Properties](#Queue_Properties)
+    * [Other Properties](#Other_Properties)
+    * [Reviewing the configuration of the CapacityScheduler](#Reviewing_the_configuration_of_the_CapacityScheduler)
+* [Changing Queue Configuration](#Changing_Queue_Configuration)
+
+Purpose
+-------
+
+This document describes the `CapacityScheduler`, a pluggable scheduler for Hadoop which allows for multiple-tenants to securely share a large cluster such that their applications are allocated resources in a timely manner under constraints of allocated capacities.
+
+Overview
+--------
+
+The `CapacityScheduler` is designed to run Hadoop applications as a shared, multi-tenant cluster in an operator-friendly manner while maximizing the throughput and the utilization of the cluster.
+
+Traditionally each organization has it own private set of compute resources that have sufficient capacity to meet the organization's SLA under peak or near peak conditions. This generally leads to poor average utilization and overhead of managing multiple independent clusters, one per each organization. Sharing clusters between organizations is a cost-effective manner of running large Hadoop installations since this allows them to reap benefits of economies of scale without creating private clusters. However, organizations are concerned about sharing a cluster because they are worried about others using the resources that are critical for their SLAs.
+
+The `CapacityScheduler` is designed to allow sharing a large cluster while giving each organization capacity guarantees. The central idea is that the available resources in the Hadoop cluster are shared among multiple organizations who collectively fund the cluster based on their computing needs. There is an added benefit that an organization can access any excess capacity not being used by others. This provides elasticity for the organizations in a cost-effective manner.
+
+Sharing clusters across organizations necessitates strong support for multi-tenancy since each organization must be guaranteed capacity and safe-guards to ensure the shared cluster is impervious to single rouge application or user or sets thereof. The `CapacityScheduler` provides a stringent set of limits to ensure that a single application or user or queue cannot consume disproportionate amount of resources in the cluster. Also, the `CapacityScheduler` provides limits on initialized/pending applications from a single user and queue to ensure fairness and stability of the cluster.
+
+The primary abstraction provided by the `CapacityScheduler` is the concept of *queues*. These queues are typically setup by administrators to reflect the economics of the shared cluster.
+
+To provide further control and predictability on sharing of resources, the `CapacityScheduler` supports *hierarchical queues* to ensure resources are shared among the sub-queues of an organization before other queues are allowed to use free resources, there-by providing *affinity* for sharing free resources among applications of a given organization.
+
+Features
+--------
+
+The `CapacityScheduler` supports the following features:
+
+* **Hierarchical Queues** - Hierarchy of queues is supported to ensure resources are shared among the sub-queues of an organization before other queues are allowed to use free resources, there-by providing more control and predictability.
+
+* **Capacity Guarantees** - Queues are allocated a fraction of the capacity of the grid in the sense that a certain capacity of resources will be at their disposal. All applications submitted to a queue will have access to the capacity allocated to the queue. Adminstrators can configure soft limits and optional hard limits on the capacity allocated to each queue.
+
+* **Security** - Each queue has strict ACLs which controls which users can submit applications to individual queues. Also, there are safe-guards to ensure that users cannot view and/or modify applications from other users. Also, per-queue and system administrator roles are supported.
+
+* **Elasticity** - Free resources can be allocated to any queue beyond it's capacity. When there is demand for these resources from queues running below capacity at a future point in time, as tasks scheduled on these resources complete, they will be assigned to applications on queues running below the capacity (pre-emption is not supported). This ensures that resources are available in a predictable and elastic manner to queues, thus preventing artifical silos of resources in the cluster which helps utilization.
+
+* **Multi-tenancy** - Comprehensive set of limits are provided to prevent a single application, user and queue from monopolizing resources of the queue or the cluster as a whole to ensure that the cluster isn't overwhelmed.
+
+* **Operability**
+
+    * Runtime Configuration - The queue definitions and properties such as capacity, ACLs can be changed, at runtime, by administrators in a secure manner to minimize disruption to users. Also, a console is provided for users and administrators to view current allocation of resources to various queues in the system. Administrators can *add additional queues* at runtime, but queues cannot be *deleted* at runtime.
+
+    * Drain applications - Administrators can *stop* queues at runtime to ensure that while existing applications run to completion, no new applications can be submitted. If a queue is in `STOPPED` state, new applications cannot be submitted to *itself* or *any of its child queueus*. Existing applications continue to completion, thus the queue can be *drained* gracefully. Administrators can also *start* the stopped queues.
+
+* **Resource-based Scheduling** - Support for resource-intensive applications, where-in a application can optionally specify higher resource-requirements than the default, there-by accomodating applications with differing resource requirements. Currently, *memory* is the the resource requirement supported.
+
+Configuration
+-------------
+
+###Setting up `ResourceManager` to use `CapacityScheduler`
+
+  To configure the `ResourceManager` to use the `CapacityScheduler`, set the following property in the **conf/yarn-site.xml**:
+
+| Property | Value |
+|:---- |:---- |
+| `yarn.resourcemanager.scheduler.class` | `org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler` |
+
+###Setting up queues
+
+  `etc/hadoop/capacity-scheduler.xml` is the configuration file for the `CapacityScheduler`.
+
+  The `CapacityScheduler` has a pre-defined queue called *root*. All queueus in the system are children of the root queue.
+
+  Further queues can be setup by configuring `yarn.scheduler.capacity.root.queues` with a list of comma-separated child queues.
+
+  The configuration for `CapacityScheduler` uses a concept called *queue path* to configure the hierarchy of queues. The *queue path* is the full path of the queue's hierarchy, starting at *root*, with . (dot) as the delimiter.
+
+  A given queue's children can be defined with the configuration knob: `yarn.scheduler.capacity.<queue-path>.queues`. Children do not inherit properties directly from the parent unless otherwise noted.
+
+  Here is an example with three top-level child-queues `a`, `b` and `c` and some sub-queues for `a` and `b`:
+    
+```xml
+<property>
+  <name>yarn.scheduler.capacity.root.queues</name>
+  <value>a,b,c</value>
+  <description>The queues at the this level (root is the root queue).
+  </description>
+</property>
+
+<property>
+  <name>yarn.scheduler.capacity.root.a.queues</name>
+  <value>a1,a2</value>
+  <description>The queues at the this level (root is the root queue).
+  </description>
+</property>
+
+<property>
+  <name>yarn.scheduler.capacity.root.b.queues</name>
+  <value>b1,b2,b3</value>
+  <description>The queues at the this level (root is the root queue).
+  </description>
+</property>
+```
+
+###Queue Properties
+
+  * Resource Allocation
+
+| Property | Description |
+|:---- |:---- |
+| `yarn.scheduler.capacity.<queue-path>.capacity` | Queue *capacity* in percentage (%) as a float (e.g. 12.5). The sum of capacities for all queues, at each level, must be equal to 100. Applications in the queue may consume more resources than the queue's capacity if there are free resources, providing elasticity. |
+| `yarn.scheduler.capacity.<queue-path>.maximum-capacity` | Maximum queue capacity in percentage (%) as a float. This limits the *elasticity* for applications in the queue. Defaults to -1 which disables it. |
+| `yarn.scheduler.capacity.<queue-path>.minimum-user-limit-percent` | Each queue enforces a limit on the percentage of resources allocated to a user at any given time, if there is demand for resources. The user limit can vary between a minimum and maximum value. The the former (the minimum value) is set to this property value and the latter (the maximum value) depends on the number of users who have submitted applications. For e.g., suppose the value of this property is 25. If two users have submitted applications to a queue, no single user can use more than 50% of the queue resources. If a third user submits an application, no single user can use more than 33% of the queue resources. With 4 or more users, no user can use more than 25% of the queues resources. A value of 100 implies no user limits are imposed. The default is 100. Value is specified as a integer. |
+| `yarn.scheduler.capacity.<queue-path>.user-limit-factor` | The multiple of the queue capacity which can be configured to allow a single user to acquire more resources. By default this is set to 1 which ensures that a single user can never take more than the queue's configured capacity irrespective of how idle th cluster is. Value is specified as a float. |
+| `yarn.scheduler.capacity.<queue-path>.maximum-allocation-mb` | The per queue maximum limit of memory to allocate to each container request at the Resource Manager. This setting overrides the cluster configuration `yarn.scheduler.maximum-allocation-mb`. This value must be smaller than or equal to the cluster maximum. |
+| `yarn.scheduler.capacity.<queue-path>.maximum-allocation-vcores` | The per queue maximum limit of virtual cores to allocate to each container request at the Resource Manager. This setting overrides the cluster configuration `yarn.scheduler.maximum-allocation-vcores`. This value must be smaller than or equal to the cluster maximum. |
+
+  * Running and Pending Application Limits
+  
+  The `CapacityScheduler` supports the following parameters to control the running and pending applications:
+
+| Property | Description |
+|:---- |:---- |
+| `yarn.scheduler.capacity.maximum-applications` / `yarn.scheduler.capacity.<queue-path>.maximum-applications` | Maximum number of applications in the system which can be concurrently active both running and pending. Limits on each queue are directly proportional to their queue capacities and user limits. This is a hard limit and any applications submitted when this limit is reached will be rejected. Default is 10000. This can be set for all queues with `yarn.scheduler.capacity.maximum-applications` and can also be overridden on a per queue basis by setting `yarn.scheduler.capacity.<queue-path>.maximum-applications`. Integer value expected. |
+| `yarn.scheduler.capacity.maximum-am-resource-percent` / `yarn.scheduler.capacity.<queue-path>.maximum-am-resource-percent` | Maximum percent of resources in the cluster which can be used to run application masters - controls number of concurrent active applications. Limits on each queue are directly proportional to their queue capacities and user limits. Specified as a float - ie 0.5 = 50%. Default is 10%. This can be set for all queues with `yarn.scheduler.capacity.maximum-am-resource-percent` and can also be overridden on a per queue basis by setting `yarn.scheduler.capacity.<queue-path>.maximum-am-resource-percent` |
+
+  * Queue Administration & Permissions
+  
+  The `CapacityScheduler` supports the following parameters to the administer the queues:
+
+| Property | Description |
+|:---- |:---- |
+| `yarn.scheduler.capacity.<queue-path>.state` | The *state* of the queue. Can be one of `RUNNING` or `STOPPED`. If a queue is in `STOPPED` state, new applications cannot be submitted to *itself* or *any of its child queues*. Thus, if the *root* queue is `STOPPED` no applications can be submitted to the entire cluster. Existing applications continue to completion, thus the queue can be *drained* gracefully. Value is specified as Enumeration. |
+| `yarn.scheduler.capacity.root.<queue-path>.acl_submit_applications` | The *ACL* which controls who can *submit* applications to the given queue. If the given user/group has necessary ACLs on the given queue or *one of the parent queues in the hierarchy* they can submit applications. *ACLs* for this property *are* inherited from the parent queue if not specified. |
+| `yarn.scheduler.capacity.root.<queue-path>.acl_administer_queue` | The *ACL* which controls who can *administer* applications on the given queue. If the given user/group has necessary ACLs on the given queue or *one of the parent queues in the hierarchy* they can administer applications. *ACLs* for this property *are* inherited from the parent queue if not specified. |
+
+**Note:** An *ACL* is of the form *user1*, *user2spacegroup1*, *group2*. The special value of * implies *anyone*. The special value of *space* implies *no one*. The default is * for the root queue if not specified.
+
+###Other Properties
+
+  * Resource Calculator
+
+| Property | Description |
+|:---- |:---- |
+| `yarn.scheduler.capacity.resource-calculator` | The ResourceCalculator implementation to be used to compare Resources in the scheduler. The default i.e. org.apache.hadoop.yarn.util.resource.DefaultResourseCalculator only uses Memory while DominantResourceCalculator uses Dominant-resource to compare multi-dimensional resources such as Memory, CPU etc. A Java ResourceCalculator class name is expected. |
+
+  * Data Locality
+
+| Property | Description |
+|:---- |:---- |
+| `yarn.scheduler.capacity.node-locality-delay` | Number of missed scheduling opportunities after which the CapacityScheduler attempts to schedule rack-local containers. Typically, this should be set to number of nodes in the cluster. By default is setting approximately number of nodes in one rack which is 40. Positive integer value is expected. |
+
+###Reviewing the configuration of the CapacityScheduler
+
+  Once the installation and configuration is completed, you can review it after starting the YARN cluster from the web-ui.
+
+  * Start the YARN cluster in the normal manner.
+
+  * Open the `ResourceManager` web UI.
+
+  * The */scheduler* web-page should show the resource usages of individual queues.
+
+Changing Queue Configuration
+----------------------------
+
+Changing queue properties and adding new queues is very simple. You need to edit **conf/capacity-scheduler.xml** and run *yarn rmadmin -refreshQueues*.
+
+    $ vi $HADOOP_CONF_DIR/capacity-scheduler.xml
+    $ $HADOOP_YARN_HOME/bin/yarn rmadmin -refreshQueues
+
+**Note:** Queues cannot be *deleted*, only addition of new queues is supported - the updated queue configuration should be a valid one i.e. queue-capacity at each *level* should be equal to 100%.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/06aca7c6/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/DockerContainerExecutor.md.vm
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/DockerContainerExecutor.md.vm b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/DockerContainerExecutor.md.vm
new file mode 100644
index 0000000..fbfe04b
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/DockerContainerExecutor.md.vm
@@ -0,0 +1,154 @@
+<!---
+  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.
+-->
+
+Docker Container Executor
+=========================
+
+* [Overview](#Overview)
+* [Cluster Configuration](#Cluster_Configuration)
+* [Tips for connecting to a secure docker repository](#Tips_for_connecting_to_a_secure_docker_repository)
+* [Job Configuration](#Job_Configuration)
+* [Docker Image Requirements](#Docker_Image_Requirements)
+* [Working example of yarn launched docker containers](#Working_example_of_yarn_launched_docker_containers)
+
+Overview
+--------
+
+[Docker](https://www.docker.io/) combines an easy-to-use interface to Linux containers with easy-to-construct image files for those containers. In short, Docker launches very light weight virtual machines.
+
+The Docker Container Executor (DCE) allows the YARN NodeManager to launch YARN containers into Docker containers. Users can specify the Docker images they want for their YARN containers. These containers provide a custom software environment in which the user's code runs, isolated from the software environment of the NodeManager. These containers can include special libraries needed by the application, and they can have different versions of Perl, Python, and even Java than what is installed on the NodeManager. Indeed, these containers can run a different flavor of Linux than what is running on the NodeManager -- although the YARN container must define all the environments and libraries needed to run the job, nothing will be shared with the NodeManager.
+
+Docker for YARN provides both consistency (all YARN containers will have the same software environment) and isolation (no interference with whatever is installed on the physical machine).
+
+Cluster Configuration
+---------------------
+
+Docker Container Executor runs in non-secure mode of HDFS and YARN. It will not run in secure mode, and will exit if it detects secure mode.
+
+The DockerContainerExecutor requires Docker daemon to be running on the NodeManagers, and the Docker client installed and able to start Docker containers. To prevent timeouts while starting jobs, the Docker images to be used by a job should already be downloaded in the NodeManagers. Here's an example of how this can be done:
+
+    sudo docker pull sequenceiq/hadoop-docker:2.4.1
+
+This should be done as part of the NodeManager startup.
+
+The following properties must be set in yarn-site.xml:
+
+```xml
+<property>
+ <name>yarn.nodemanager.docker-container-executor.exec-name</name>
+  <value>/usr/bin/docker</value>
+  <description>
+     Name or path to the Docker client. This is a required parameter. If this is empty,
+     user must pass an image name as part of the job invocation(see below).
+  </description>
+</property>
+
+<property>
+  <name>yarn.nodemanager.container-executor.class</name>
+  <value>org.apache.hadoop.yarn.server.nodemanager.DockerContainerExecutor</value>
+  <description>
+     This is the container executor setting that ensures that all
+jobs are started with the DockerContainerExecutor.
+  </description>
+</property>
+```
+
+Administrators should be aware that DCE doesn't currently provide user name-space isolation. This means, in particular, that software running as root in the YARN container will have root privileges in the underlying NodeManager. Put differently, DCE currently provides no better security guarantees than YARN's Default Container Executor. In fact, DockerContainerExecutor will exit if it detects secure yarn.
+
+Tips for connecting to a secure docker repository
+-------------------------------------------------
+
+By default, docker images are pulled from the docker public repository. The format of a docker image url is: *username*/*image\_name*. For example, sequenceiq/hadoop-docker:2.4.1 is an image in docker public repository that contains java and hadoop.
+
+If you want your own private repository, you provide the repository url instead of your username. Therefore, the image url becomes: *private\_repo\_url*/*image\_name*. For example, if your repository is on localhost:8080, your images would be like: localhost:8080/hadoop-docker
+
+To connect to a secure docker repository, you can use the following invocation:
+
+```
+    docker login [OPTIONS] [SERVER]
+
+    Register or log in to a Docker registry server, if no server is specified
+    "https://index.docker.io/v1/" is the default.
+
+  -e, --email=""       Email
+  -p, --password=""    Password
+  -u, --username=""    Username
+```
+
+If you want to login to a self-hosted registry you can specify this by adding the server name.
+
+    docker login <private_repo_url>
+
+This needs to be run as part of the NodeManager startup, or as a cron job if the login session expires periodically. You can login to multiple docker repositories from the same NodeManager, but all your users will have access to all your repositories, as at present the DockerContainerExecutor does not support per-job docker login.
+
+Job Configuration
+-----------------
+
+Currently you cannot configure any of the Docker settings with the job configuration. You can provide Mapper, Reducer, and ApplicationMaster environment overrides for the docker images, using the following 3 JVM properties respectively(only for MR jobs):
+
+* `mapreduce.map.env`: You can override the mapper's image by passing `yarn.nodemanager.docker-container-executor.image-name`=*your_image_name* to this JVM property.
+
+* `mapreduce.reduce.env`: You can override the reducer's image by passing `yarn.nodemanager.docker-container-executor.image-name`=*your_image_name* to this JVM property.
+
+* `yarn.app.mapreduce.am.env`: You can override the ApplicationMaster's image by passing `yarn.nodemanager.docker-container-executor.image-name`=*your_image_name* to this JVM property.
+
+Docker Image Requirements
+-------------------------
+
+The Docker Images used for YARN containers must meet the following requirements:
+
+The distro and version of Linux in your Docker Image can be quite different from that of your NodeManager. (Docker does have a few limitations in this regard, but you're not likely to hit them.) However, if you're using the MapReduce framework, then your image will need to be configured for running Hadoop. Java must be installed in the container, and the following environment variables must be defined in the image: JAVA_HOME, HADOOP_COMMON_PATH, HADOOP_HDFS_HOME, HADOOP_MAPRED_HOME, HADOOP_YARN_HOME, and HADOOP_CONF_DIR
+
+Working example of yarn launched docker containers
+--------------------------------------------------
+
+The following example shows how to run teragen using DockerContainerExecutor.
+
+Step 1. First ensure that YARN is properly configured with DockerContainerExecutor(see above).
+
+```xml
+<property>
+ <name>yarn.nodemanager.docker-container-executor.exec-name</name>
+  <value>docker -H=tcp://0.0.0.0:4243</value>
+  <description>
+     Name or path to the Docker client. The tcp socket must be
+     where docker daemon is listening.
+  </description>
+</property>
+
+<property>
+  <name>yarn.nodemanager.container-executor.class</name>
+  <value>org.apache.hadoop.yarn.server.nodemanager.DockerContainerExecutor</value>
+  <description>
+     This is the container executor setting that ensures that all
+jobs are started with the DockerContainerExecutor.
+  </description>
+</property>
+```
+
+Step 2. Pick a custom Docker image if you want. In this example, we'll use sequenceiq/hadoop-docker:2.4.1 from the docker hub repository. It has jdk, hadoop, and all the previously mentioned environment variables configured.
+
+Step 3. Run.
+
+```bash
+hadoop jar $HADOOP_PREFIX/share/hadoop/mapreduce/hadoop-mapreduce-examples-${project.version}.jar \
+  teragen \
+     -Dmapreduce.map.env="yarn.nodemanager.docker-container-executor.image-name=sequenceiq/hadoop-docker:2.4.1" \
+   -Dyarn.app.mapreduce.am.env="yarn.nodemanager.docker-container-executor.image-name=sequenceiq/hadoop-docker:2.4.1" \
+  1000 \
+  teragen_out_dir
+```
+
+  Once it succeeds, you can check the yarn debug logs to verify that docker indeed has launched containers.
+


[32/50] [abbrv] hadoop git commit: YARN-3168. Convert site documentation from apt to markdown (Gururaj Shetty via aw)

Posted by zh...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/06aca7c6/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/FairScheduler.md
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/FairScheduler.md b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/FairScheduler.md
new file mode 100644
index 0000000..1812a44
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/FairScheduler.md
@@ -0,0 +1,233 @@
+<!---
+  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: Fair Scheduler
+======================
+
+* [Purpose](#Purpose)
+* [Introduction](#Introduction)
+* [Hierarchical queues with pluggable policies](#Hierarchical_queues_with_pluggable_policies)
+* [Automatically placing applications in queues](#Automatically_placing_applications_in_queues)
+* [Installation](#Installation)
+* [Configuration](#Configuration)
+    * [Properties that can be placed in yarn-site.xml](#Properties_that_can_be_placed_in_yarn-site.xml)
+    * [Allocation file format](#Allocation_file_format)
+    * [Queue Access Control Lists](#Queue_Access_Control_Lists)
+* [Administration](#Administration)
+    * [Modifying configuration at runtime](#Modifying_configuration_at_runtime)
+    * [Monitoring through web UI](#Monitoring_through_web_UI)
+    * [Moving applications between queues](#Moving_applications_between_queues)
+
+##Purpose
+
+This document describes the `FairScheduler`, a pluggable scheduler for Hadoop that allows YARN applications to share resources in large clusters fairly.
+
+##Introduction
+
+Fair scheduling is a method of assigning resources to applications such that all apps get, on average, an equal share of resources over time. Hadoop NextGen is capable of scheduling multiple resource types. By default, the Fair Scheduler bases scheduling fairness decisions only on memory. It can be configured to schedule with both memory and CPU, using the notion of Dominant Resource Fairness developed by Ghodsi et al. When there is a single app running, that app uses the entire cluster. When other apps are submitted, resources that free up are assigned to the new apps, so that each app eventually on gets roughly the same amount of resources. Unlike the default Hadoop scheduler, which forms a queue of apps, this lets short apps finish in reasonable time while not starving long-lived apps. It is also a reasonable way to share a cluster between a number of users. Finally, fair sharing can also work with app priorities - the priorities are used as weights to determine the fraction of t
 otal resources that each app should get.
+
+The scheduler organizes apps further into "queues", and shares resources fairly between these queues. By default, all users share a single queue, named "default". If an app specifically lists a queue in a container resource request, the request is submitted to that queue. It is also possible to assign queues based on the user name included with the request through configuration. Within each queue, a scheduling policy is used to share resources between the running apps. The default is memory-based fair sharing, but FIFO and multi-resource with Dominant Resource Fairness can also be configured. Queues can be arranged in a hierarchy to divide resources and configured with weights to share the cluster in specific proportions.
+
+In addition to providing fair sharing, the Fair Scheduler allows assigning guaranteed minimum shares to queues, which is useful for ensuring that certain users, groups or production applications always get sufficient resources. When a queue contains apps, it gets at least its minimum share, but when the queue does not need its full guaranteed share, the excess is split between other running apps. This lets the scheduler guarantee capacity for queues while utilizing resources efficiently when these queues don't contain applications.
+
+The Fair Scheduler lets all apps run by default, but it is also possible to limit the number of running apps per user and per queue through the config file. This can be useful when a user must submit hundreds of apps at once, or in general to improve performance if running too many apps at once would cause too much intermediate data to be created or too much context-switching. Limiting the apps does not cause any subsequently submitted apps to fail, only to wait in the scheduler's queue until some of the user's earlier apps finish.
+
+##Hierarchical queues with pluggable policies
+
+The fair scheduler supports hierarchical queues. All queues descend from a queue named "root". Available resources are distributed among the children of the root queue in the typical fair scheduling fashion. Then, the children distribute the resources assigned to them to their children in the same fashion. Applications may only be scheduled on leaf queues. Queues can be specified as children of other queues by placing them as sub-elements of their parents in the fair scheduler allocation file.
+
+A queue's name starts with the names of its parents, with periods as separators. So a queue named "queue1" under the root queue, would be referred to as "root.queue1", and a queue named "queue2" under a queue named "parent1" would be referred to as "root.parent1.queue2". When referring to queues, the root part of the name is optional, so queue1 could be referred to as just "queue1", and a queue2 could be referred to as just "parent1.queue2".
+
+Additionally, the fair scheduler allows setting a different custom policy for each queue to allow sharing the queue's resources in any which way the user wants. A custom policy can be built by extending `org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.SchedulingPolicy`. FifoPolicy, FairSharePolicy (default), and DominantResourceFairnessPolicy are built-in and can be readily used.
+
+Certain add-ons are not yet supported which existed in the original (MR1) Fair Scheduler. Among them, is the use of a custom policies governing priority "boosting" over certain apps.
+
+##Automatically placing applications in queues
+
+The Fair Scheduler allows administrators to configure policies that automatically place submitted applications into appropriate queues. Placement can depend on the user and groups of the submitter and the requested queue passed by the application. A policy consists of a set of rules that are applied sequentially to classify an incoming application. Each rule either places the app into a queue, rejects it, or continues on to the next rule. Refer to the allocation file format below for how to configure these policies.
+
+##Installation
+
+To use the Fair Scheduler first assign the appropriate scheduler class in yarn-site.xml:
+
+    <property>
+      <name>yarn.resourcemanager.scheduler.class</name>
+      <value>org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.FairScheduler</value>
+    </property>
+
+##Configuration
+
+Customizing the Fair Scheduler typically involves altering two files. First, scheduler-wide options can be set by adding configuration properties in the yarn-site.xml file in your existing configuration directory. Second, in most cases users will want to create an allocation file listing which queues exist and their respective weights and capacities. The allocation file is reloaded every 10 seconds, allowing changes to be made on the fly.
+
+###Properties that can be placed in yarn-site.xml
+
+| Property | Description |
+|:---- |:---- |
+| `yarn.scheduler.fair.allocation.file` | Path to allocation file. An allocation file is an XML manifest describing queues and their properties, in addition to certain policy defaults. This file must be in the XML format described in the next section. If a relative path is given, the file is searched for on the classpath (which typically includes the Hadoop conf directory). Defaults to fair-scheduler.xml. |
+| `yarn.scheduler.fair.user-as-default-queue` | Whether to use the username associated with the allocation as the default queue name, in the event that a queue name is not specified. If this is set to "false" or unset, all jobs have a shared default queue, named "default". Defaults to true. If a queue placement policy is given in the allocations file, this property is ignored. |
+| `yarn.scheduler.fair.preemption` | Whether to use preemption. Defaults to false. |
+| `yarn.scheduler.fair.preemption.cluster-utilization-threshold` | The utilization threshold after which preemption kicks in. The utilization is computed as the maximum ratio of usage to capacity among all resources. Defaults to 0.8f. |
+| `yarn.scheduler.fair.sizebasedweight` | Whether to assign shares to individual apps based on their size, rather than providing an equal share to all apps regardless of size. When set to true, apps are weighted by the natural logarithm of one plus the app's total requested memory, divided by the natural logarithm of 2. Defaults to false. |
+| `yarn.scheduler.fair.assignmultiple` | Whether to allow multiple container assignments in one heartbeat. Defaults to false. |
+| `yarn.scheduler.fair.max.assign` | If assignmultiple is true, the maximum amount of containers that can be assigned in one heartbeat. Defaults to -1, which sets no limit. |
+| `yarn.scheduler.fair.locality.threshold.node` | For applications that request containers on particular nodes, the number of scheduling opportunities since the last container assignment to wait before accepting a placement on another node. Expressed as a float between 0 and 1, which, as a fraction of the cluster size, is the number of scheduling opportunities to pass up. The default value of -1.0 means don't pass up any scheduling opportunities. |
+| `yarn.scheduler.fair.locality.threshold.rack` | For applications that request containers on particular racks, the number of scheduling opportunities since the last container assignment to wait before accepting a placement on another rack. Expressed as a float between 0 and 1, which, as a fraction of the cluster size, is the number of scheduling opportunities to pass up. The default value of -1.0 means don't pass up any scheduling opportunities. |
+| `yarn.scheduler.fair.allow-undeclared-pools` | If this is true, new queues can be created at application submission time, whether because they are specified as the application's queue by the submitter or because they are placed there by the user-as-default-queue property. If this is false, any time an app would be placed in a queue that is not specified in the allocations file, it is placed in the "default" queue instead. Defaults to true. If a queue placement policy is given in the allocations file, this property is ignored. |
+| `yarn.scheduler.fair.update-interval-ms` | The interval at which to lock the scheduler and recalculate fair shares, recalculate demand, and check whether anything is due for preemption. Defaults to 500 ms. |
+
+###Allocation file format
+
+The allocation file must be in XML format. The format contains five types of elements:
+
+* **Queue elements**: which represent queues. Queue elements can take an optional attribute 'type', which when set to 'parent' makes it a parent queue. This is useful when we want to create a parent queue without configuring any leaf queues. Each queue element may contain the following properties:
+
+    * minResources: minimum resources the queue is entitled to, in the form "X mb, Y vcores". For the single-resource fairness policy, the vcores value is ignored. If a queue's minimum share is not satisfied, it will be offered available resources before any other queue under the same parent. Under the single-resource fairness policy, a queue is considered unsatisfied if its memory usage is below its minimum memory share. Under dominant resource fairness, a queue is considered unsatisfied if its usage for its dominant resource with respect to the cluster capacity is below its minimum share for that resource. If multiple queues are unsatisfied in this situation, resources go to the queue with the smallest ratio between relevant resource usage and minimum. Note that it is possible that a queue that is below its minimum may not immediately get up to its minimum when it submits an application, because already-running jobs may be using those resources.
+
+    * maxResources: maximum resources a queue is allowed, in the form "X mb, Y vcores". For the single-resource fairness policy, the vcores value is ignored. A queue will never be assigned a container that would put its aggregate usage over this limit.
+
+    * maxRunningApps: limit the number of apps from the queue to run at once
+
+    * maxAMShare: limit the fraction of the queue's fair share that can be used to run application masters. This property can only be used for leaf queues. For example, if set to 1.0f, then AMs in the leaf queue can take up to 100% of both the memory and CPU fair share. The value of -1.0f will disable this feature and the amShare will not be checked. The default value is 0.5f.
+
+    * weight: to share the cluster non-proportionally with other queues. Weights default to 1, and a queue with weight 2 should receive approximately twice as many resources as a queue with the default weight.
+
+    * schedulingPolicy: to set the scheduling policy of any queue. The allowed values are "fifo"/"fair"/"drf" or any class that extends `org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.SchedulingPolicy`. Defaults to "fair". If "fifo", apps with earlier submit times are given preference 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 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. If not set, the queue will inherit the value from its parent queue.
+
+    * fairSharePreemptionTimeout: number of seconds the queue is under its fair share threshold before it will try to preempt containers to take resources from other queues. If not set, the queue will inherit the value from its parent queue.
+
+    * fairSharePreemptionThreshold: the fair share preemption threshold for the queue. If the queue waits fairSharePreemptionTimeout without receiving fairSharePreemptionThreshold\*fairShare resources, it is allowed to preempt containers to take resources from other queues. If not set, the queue will inherit the value from its parent queue.
+
+* **User elements**: which represent settings governing the behavior of individual users. They can contain a single property: maxRunningApps, a limit on the number of running apps for a particular user.
+
+* **A userMaxAppsDefault element**: which sets the default running app limit for any users whose limit is not otherwise specified.
+
+* **A defaultFairSharePreemptionTimeout element**: which sets the fair share preemption timeout for the root queue; overridden by fairSharePreemptionTimeout element in root queue.
+
+* **A defaultMinSharePreemptionTimeout element**: which sets the min share preemption timeout for the root queue; overridden by minSharePreemptionTimeout element in root queue.
+
+* **A defaultFairSharePreemptionThreshold element**: which sets the fair share preemption threshold for the root queue; overridden by fairSharePreemptionThreshold element in root queue.
+
+* **A queueMaxAppsDefault element**: which sets the default running app limit for queues; overriden by maxRunningApps element in each queue.
+
+* **A queueMaxAMShareDefault element**: which sets the default AM resource limit for queue; overriden by maxAMShare element in each queue.
+
+* **A defaultQueueSchedulingPolicy element**: which sets the default scheduling policy for queues; overriden by the schedulingPolicy element in each queue if specified. Defaults to "fair".
+
+* **A queuePlacementPolicy element**: which contains a list of rule elements that tell the scheduler how to place incoming apps into queues. Rules are applied in the order that they are listed. Rules may take arguments. All rules accept the "create" argument, which indicates whether the rule can create a new queue. "Create" defaults to true; if set to false and the rule would place the app in a queue that is not configured in the allocations file, we continue on to the next rule. The last rule must be one that can never issue a continue. Valid rules are:
+
+    * specified: the app is placed into the queue it requested. If the app requested no queue, i.e. it specified "default", we continue. If the app requested a queue name starting or ending with period, i.e. names like ".q1" or "q1." will be rejected.
+
+    * user: the app is placed into a queue with the name of the user who submitted it. Periods in the username will be replace with "\_dot\_", i.e. the queue name for user "first.last" is "first\_dot\_last".
+
+    * primaryGroup: the app is placed into a queue with the name of the primary group of the user who submitted it. Periods in the group name will be replaced with "\_dot\_", i.e. the queue name for group "one.two" is "one\_dot\_two".
+
+    * secondaryGroupExistingQueue: the app is placed into a queue with a name that matches a secondary group of the user who submitted it. The first secondary group that matches a configured queue will be selected. Periods in group names will be replaced with "\_dot\_", i.e. a user with "one.two" as one of their secondary groups would be placed into the "one\_dot\_two" queue, if such a queue exists.
+
+    * nestedUserQueue : the app is placed into a queue with the name of the user under the queue suggested by the nested rule. This is similar to ‘user’ rule,the difference being in 'nestedUserQueue' rule,user queues can be created under any parent queue, while 'user' rule creates user queues only under root queue. Note that nestedUserQueue rule would be applied only if the nested rule returns a parent queue.One can configure a parent queue either by setting 'type' attribute of queue to 'parent' or by configuring at least one leaf under that queue which makes it a parent. See example allocation for a sample use case.
+
+    * default: the app is placed into the queue specified in the 'queue' attribute of the default rule. If 'queue' attribute is not specified, the app is placed into 'root.default' queue.
+
+    * reject: the app is rejected.
+
+    An example allocation file is given here:
+
+```xml
+<?xml version="1.0"?>
+<allocations>
+  <queue name="sample_queue">
+    <minResources>10000 mb,0vcores</minResources>
+    <maxResources>90000 mb,0vcores</maxResources>
+    <maxRunningApps>50</maxRunningApps>
+    <maxAMShare>0.1</maxAMShare>
+    <weight>2.0</weight>
+    <schedulingPolicy>fair</schedulingPolicy>
+    <queue name="sample_sub_queue">
+      <aclSubmitApps>charlie</aclSubmitApps>
+      <minResources>5000 mb,0vcores</minResources>
+    </queue>
+  </queue>
+
+  <queueMaxAMShareDefault>0.5</queueMaxAMShareDefault>
+
+  <!-- Queue 'secondary_group_queue' is a parent queue and may have
+       user queues under it -->
+  <queue name="secondary_group_queue" type="parent">
+  <weight>3.0</weight>
+  </queue>
+  
+  <user name="sample_user">
+    <maxRunningApps>30</maxRunningApps>
+  </user>
+  <userMaxAppsDefault>5</userMaxAppsDefault>
+  
+  <queuePlacementPolicy>
+    <rule name="specified" />
+    <rule name="primaryGroup" create="false" />
+    <rule name="nestedUserQueue">
+        <rule name="secondaryGroupExistingQueue" create="false" />
+    </rule>
+    <rule name="default" queue="sample_queue"/>
+  </queuePlacementPolicy>
+</allocations>
+```
+
+  Note that for backwards compatibility with the original FairScheduler, "queue" elements can instead be named as "pool" elements.
+
+###Queue Access Control Lists
+
+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.
+
+**Note:** The delimiter is a space character. To specify only ACL groups, begin the value with a space character.
+
+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 "\*".
+
+##Administration
+
+The fair scheduler provides support for administration at runtime through a few mechanisms:
+
+###Modifying configuration at runtime
+
+It is possible to modify minimum shares, limits, weights, preemption timeouts and queue scheduling policies at runtime by editing the allocation file. The scheduler will reload this file 10-15 seconds after it sees that it was modified.
+
+###Monitoring through web UI
+
+Current applications, queues, and fair shares can be examined through the ResourceManager's web interface, at `http://*ResourceManager URL*/cluster/scheduler`.
+
+The following fields can be seen for each queue on the web interface:
+
+* Used Resources - The sum of resources allocated to containers within the queue.
+
+* Num Active Applications - The number of applications in the queue that have received at least one container.
+
+* Num Pending Applications - The number of applications in the queue that have not yet received any containers.
+
+* Min Resources - The configured minimum resources that are guaranteed to the queue.
+
+* Max Resources - The configured maximum resources that are allowed to the queue.
+
+* Instantaneous Fair Share - The queue's instantaneous fair share of resources. These shares consider only actives queues (those with running applications), and are used for scheduling decisions. Queues may be allocated resources beyond their shares when other queues aren't using them. A queue whose resource consumption lies at or below its instantaneous fair share will never have its containers preempted.
+
+* Steady Fair Share - The queue's steady fair share of resources. These shares consider all the queues irrespective of whether they are active (have running applications) or not. These are computed less frequently and change only when the configuration or capacity changes.They are meant to provide visibility into resources the user can expect, and hence displayed in the Web UI.
+
+###Moving applications between queues
+
+The Fair Scheduler supports moving a running application to a different queue. This can be useful for moving an important application to a higher priority queue, or for moving an unimportant application to a lower priority queue. Apps can be moved by running `yarn application -movetoqueue appID -queue targetQueueName`.
+
+When an application is moved to a queue, its existing allocations become counted with the new queue's allocations instead of the old for purposes of determining fairness. An attempt to move an application to a queue will fail if the addition of the app's resources to that queue would violate the its maxRunningApps or maxResources constraints.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/06aca7c6/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/NodeManager.md
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/NodeManager.md b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/NodeManager.md
new file mode 100644
index 0000000..6341c60
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/NodeManager.md
@@ -0,0 +1,57 @@
+<!---
+  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.
+-->
+
+NodeManager Overview
+=====================
+
+* [Overview](#Overview)
+* [Health Checker Service](#Health_checker_service)
+    * [Disk Checker](#Disk_Checker)
+    * [External Health Script](#External_Health_Script)
+
+Overview
+--------
+
+The NodeManager is responsible for launching and managing containers on a node. Containers execute tasks as specified by the AppMaster.
+
+Health Checker Service
+----------------------
+
+The NodeManager runs services to determine the health of the node it is executing on. The services perform checks on the disk as well as any user specified tests. If any health check fails, the NodeManager marks the node as unhealthy and communicates this to the ResourceManager, which then stops assigning containers to the node. Communication of the node status is done as part of the heartbeat between the NodeManager and the ResourceManager. The intervals at which the disk checker and health monitor(described below) run don't affect the heartbeat intervals. When the heartbeat takes place, the status of both checks is used to determine the health of the node.
+
+###Disk Checker
+
+  The disk checker checks the state of the disks that the NodeManager is configured to use(local-dirs and log-dirs, configured using yarn.nodemanager.local-dirs and yarn.nodemanager.log-dirs respectively). The checks include permissions and free disk space. It also checks that the filesystem isn't in a read-only state. The checks are run at 2 minute intervals by default but can be configured to run as often as the user desires. If a disk fails the check, the NodeManager stops using that particular disk but still reports the node status as healthy. However if a number of disks fail the check(the number can be configured, as explained below), then the node is reported as unhealthy to the ResourceManager and new containers will not be assigned to the node. In addition, once a disk is marked as unhealthy, the NodeManager stops checking it to see if it has recovered(e.g. disk became full and was then cleaned up). The only way for the NodeManager to use that disk to restart the software o
 n the node. The following configuration parameters can be used to modify the disk checks:
+
+| Configuration Name | Allowed Values | Description |
+|:---- |:---- |:---- |
+| `yarn.nodemanager.disk-health-checker.enable` | true, false | Enable or disable the disk health checker service |
+| `yarn.nodemanager.disk-health-checker.interval-ms` | Positive integer | The interval, in milliseconds, at which the disk checker should run; the default value is 2 minutes |
+| `yarn.nodemanager.disk-health-checker.min-healthy-disks` | Float between 0-1 | The minimum fraction of disks that must pass the check for the NodeManager to mark the node as healthy; the default is 0.25 |
+| `yarn.nodemanager.disk-health-checker.max-disk-utilization-per-disk-percentage` | Float between 0-100 | The maximum percentage of disk space that may be utilized before a disk is marked as unhealthy by the disk checker service. This check is run for every disk used by the NodeManager. The default value is 100 i.e. the entire disk can be used. |
+| `yarn.nodemanager.disk-health-checker.min-free-space-per-disk-mb` | Integer | The minimum amount of free space that must be available on the disk for the disk checker service to mark the disk as healthy. This check is run for every disk used by the NodeManager. The default value is 0 i.e. the entire disk can be used. |
+
+
+###External Health Script
+
+  Users may specify their own health checker script that will be invoked by the health checker service. Users may specify a timeout as well as options to be passed to the script. If the script exits with a non-zero exit code, times out or results in an exception being thrown, the node is marked as unhealthy. Please note that if the script cannot be executed due to permissions or an incorrect path, etc, then it counts as a failure and the node will be reported as unhealthy. Please note that speifying a health check script is not mandatory. If no script is specified, only the disk checker status will be used to determine the health of the node. The following configuration parameters can be used to set the health script:
+
+| Configuration Name | Allowed Values | Description |
+|:---- |:---- |:---- |
+| `yarn.nodemanager.health-checker.interval-ms` | Postive integer | The interval, in milliseconds, at which health checker service runs; the default value is 10 minutes. |
+| `yarn.nodemanager.health-checker.script.timeout-ms` | Postive integer | The timeout for the health script that's executed; the default value is 20 minutes. |
+| `yarn.nodemanager.health-checker.script.path` | String | Absolute path to the health check script to be run. |
+| `yarn.nodemanager.health-checker.script.opts` | String | Arguments to be passed to the script when the script is executed. |
+
+

http://git-wip-us.apache.org/repos/asf/hadoop/blob/06aca7c6/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/NodeManagerCgroups.md
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/NodeManagerCgroups.md b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/NodeManagerCgroups.md
new file mode 100644
index 0000000..79a428d
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/NodeManagerCgroups.md
@@ -0,0 +1,57 @@
+<!---
+  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.
+-->
+
+Using CGroups with YARN
+=======================
+
+* [CGroups Configuration](#CGroups_configuration)
+* [CGroups and Security](#CGroups_and_security)
+
+CGroups is a mechanism for aggregating/partitioning sets of tasks, and all their future children, into hierarchical groups with specialized behaviour. CGroups is a Linux kernel feature and was merged into kernel version 2.6.24. From a YARN perspective, this allows containers to be limited in their resource usage. A good example of this is CPU usage. Without CGroups, it becomes hard to limit container CPU usage. Currently, CGroups is only used for limiting CPU usage.
+
+CGroups Configuration
+---------------------
+
+This section describes the configuration variables for using CGroups.
+
+The following settings are related to setting up CGroups. These need to be set in *yarn-site.xml*.
+
+|Configuration Name | Description |
+|:---- |:---- |
+| `yarn.nodemanager.container-executor.class` | This should be set to "org.apache.hadoop.yarn.server.nodemanager.LinuxContainerExecutor". CGroups is a Linux kernel feature and is exposed via the LinuxContainerExecutor. |
+| `yarn.nodemanager.linux-container-executor.resources-handler.class` | This should be set to "org.apache.hadoop.yarn.server.nodemanager.util.CgroupsLCEResourcesHandler". Using the LinuxContainerExecutor doesn't force you to use CGroups. If you wish to use CGroups, the resource-handler-class must be set to CGroupsLCEResourceHandler. |
+| `yarn.nodemanager.linux-container-executor.cgroups.hierarchy` | The cgroups hierarchy under which to place YARN proccesses(cannot contain commas). If yarn.nodemanager.linux-container-executor.cgroups.mount is false (that is, if cgroups have been pre-configured), then this cgroups hierarchy must already exist |
+| `yarn.nodemanager.linux-container-executor.cgroups.mount` | Whether the LCE should attempt to mount cgroups if not found - can be true or false. |
+| `yarn.nodemanager.linux-container-executor.cgroups.mount-path` | Where the LCE should attempt to mount cgroups if not found. Common locations include /sys/fs/cgroup and /cgroup; the default location can vary depending on the Linux distribution in use. This path must exist before the NodeManager is launched. Only used when the LCE resources handler is set to the CgroupsLCEResourcesHandler, and yarn.nodemanager.linux-container-executor.cgroups.mount is true. A point to note here is that the container-executor binary will try to mount the path specified + "/" + the subsystem. In our case, since we are trying to limit CPU the binary tries to mount the path specified + "/cpu" and that's the path it expects to exist. |
+| `yarn.nodemanager.linux-container-executor.group` | The Unix group of the NodeManager. It should match the setting in "container-executor.cfg". This configuration is required for validating the secure access of the container-executor binary. |
+
+The following settings are related to limiting resource usage of YARN containers:
+
+|Configuration Name | Description |
+|:---- |:---- |
+| `yarn.nodemanager.resource.percentage-physical-cpu-limit` | This setting lets you limit the cpu usage of all YARN containers. It sets a hard upper limit on the cumulative CPU usage of the containers. For example, if set to 60, the combined CPU usage of all YARN containers will not exceed 60%. |
+| `yarn.nodemanager.linux-container-executor.cgroups.strict-resource-usage` | CGroups allows cpu usage limits to be hard or soft. When this setting is true, containers cannot use more CPU usage than allocated even if spare CPU is available. This ensures that containers can only use CPU that they were allocated. When set to false, containers can use spare CPU if available. It should be noted that irrespective of whether set to true or false, at no time can the combined CPU usage of all containers exceed the value specified in "yarn.nodemanager.resource.percentage-physical-cpu-limit". |
+
+CGroups and security
+--------------------
+
+CGroups itself has no requirements related to security. However, the LinuxContainerExecutor does have some requirements. If running in non-secure mode, by default, the LCE runs all jobs as user "nobody". This user can be changed by setting "yarn.nodemanager.linux-container-executor.nonsecure-mode.local-user" to the desired user. However, it can also be configured to run jobs as the user submitting the job. In that case "yarn.nodemanager.linux-container-executor.nonsecure-mode.limit-users" should be set to false.
+
+| yarn.nodemanager.linux-container-executor.nonsecure-mode.local-user | yarn.nodemanager.linux-container-executor.nonsecure-mode.limit-users | User running jobs |
+|:---- |:---- |:---- |
+| (default) | (default) | nobody |
+| yarn | (default) | yarn |
+| yarn | false | (User submitting the job) |
+
+

http://git-wip-us.apache.org/repos/asf/hadoop/blob/06aca7c6/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/NodeManagerRest.md
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/NodeManagerRest.md b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/NodeManagerRest.md
new file mode 100644
index 0000000..acafd28
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/NodeManagerRest.md
@@ -0,0 +1,543 @@
+<!---
+  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.
+-->
+
+NodeManager REST API's
+=======================
+
+* [Overview](#Overview)
+* [NodeManager Information API](#NodeManager_Information_API)
+* [Applications API](#Applications_API)
+* [Application API](#Application_API)
+* [Containers API](#Containers_API)
+* [Container API](#Container_API)
+
+Overview
+--------
+
+The NodeManager REST API's allow the user to get status on the node and information about applications and containers running on that node.
+
+NodeManager Information API
+---------------------------
+
+The node information resource provides overall information about that particular node.
+
+### URI
+
+Both of the following URI's give you the cluster information.
+
+      * http://<nm http address:port>/ws/v1/node
+      * http://<nm http address:port>/ws/v1/node/info
+
+### HTTP Operations Supported
+
+      * GET
+
+### Query Parameters Supported
+
+      None
+
+### Elements of the *nodeInfo* object
+
+| Properties | Data Type | Description |
+|:---- |:---- |:---- |
+| id | long | The NodeManager id |
+| nodeHostName | string | The host name of the NodeManager |
+| totalPmemAllocatedContainersMB | long | The amount of physical memory allocated for use by containers in MB |
+| totalVmemAllocatedContainersMB | long | The amount of virtual memory allocated for use by containers in MB |
+| totalVCoresAllocatedContainers | long | The number of virtual cores allocated for use by containers |
+| lastNodeUpdateTime | long | The last timestamp at which the health report was received (in ms since epoch) |
+| healthReport | string | The diagnostic health report of the node |
+| nodeHealthy | boolean | true/false indicator of if the node is healthy |
+| nodeManagerVersion | string | Version of the NodeManager |
+| nodeManagerBuildVersion | string | NodeManager build string with build version, user, and checksum |
+| nodeManagerVersionBuiltOn | string | Timestamp when NodeManager was built(in ms since epoch) |
+| hadoopVersion | string | Version of hadoop common |
+| hadoopBuildVersion | string | Hadoop common build string with build version, user, and checksum |
+| hadoopVersionBuiltOn | string | Timestamp when hadoop common was built(in ms since epoch) |
+
+### Response Examples
+
+**JSON response**
+
+HTTP Request:
+
+      GET http://<nm http address:port>/ws/v1/node/info
+
+Response Header:
+
+      HTTP/1.1 200 OK
+      Content-Type: application/json
+      Transfer-Encoding: chunked
+      Server: Jetty(6.1.26)
+
+Response Body:
+
+```json
+{
+   "nodeInfo" : {
+      "hadoopVersionBuiltOn" : "Mon Jan  9 14:58:42 UTC 2012",
+      "nodeManagerBuildVersion" : "0.23.1-SNAPSHOT from 1228355 by user1 source checksum 20647f76c36430e888cc7204826a445c",
+      "lastNodeUpdateTime" : 1326222266126,
+      "totalVmemAllocatedContainersMB" : 17203,
+      "totalVCoresAllocatedContainers" : 8,
+      "nodeHealthy" : true,
+      "healthReport" : "",
+      "totalPmemAllocatedContainersMB" : 8192,
+      "nodeManagerVersionBuiltOn" : "Mon Jan  9 15:01:59 UTC 2012",
+      "nodeManagerVersion" : "0.23.1-SNAPSHOT",
+      "id" : "host.domain.com:8041",
+      "hadoopBuildVersion" : "0.23.1-SNAPSHOT from 1228292 by user1 source checksum 3eba233f2248a089e9b28841a784dd00",
+      "nodeHostName" : "host.domain.com",
+      "hadoopVersion" : "0.23.1-SNAPSHOT"
+   }
+}
+```
+
+**XML response**
+
+HTTP Request:
+
+      Accept: application/xml
+      GET http://<nm http address:port>/ws/v1/node/info
+
+Response Header:
+
+      HTTP/1.1 200 OK
+      Content-Type: application/xml
+      Content-Length: 983
+      Server: Jetty(6.1.26)
+
+Response Body:
+
+```xml
+<?xml version="1.0" encoding="UTF-8" standalone="yes"?>
+<nodeInfo>
+  <healthReport/>
+  <totalVmemAllocatedContainersMB>17203</totalVmemAllocatedContainersMB>
+  <totalPmemAllocatedContainersMB>8192</totalPmemAllocatedContainersMB>
+  <totalVCoresAllocatedContainers>8</totalVCoresAllocatedContainers>
+  <lastNodeUpdateTime>1326222386134</lastNodeUpdateTime>
+  <nodeHealthy>true</nodeHealthy>
+  <nodeManagerVersion>0.23.1-SNAPSHOT</nodeManagerVersion>
+  <nodeManagerBuildVersion>0.23.1-SNAPSHOT from 1228355 by user1 source checksum 20647f76c36430e888cc7204826a445c</nodeManagerBuildVersion>
+  <nodeManagerVersionBuiltOn>Mon Jan  9 15:01:59 UTC 2012</nodeManagerVersionBuiltOn>
+  <hadoopVersion>0.23.1-SNAPSHOT</hadoopVersion>
+  <hadoopBuildVersion>0.23.1-SNAPSHOT from 1228292 by user1 source checksum 3eba233f2248a089e9b28841a784dd00</hadoopBuildVersion>
+  <hadoopVersionBuiltOn>Mon Jan  9 14:58:42 UTC 2012</hadoopVersionBuiltOn>
+  <id>host.domain.com:8041</id>
+  <nodeHostName>host.domain.com</nodeHostName>
+</nodeInfo>
+```
+
+Applications API
+----------------
+
+With the Applications API, you can obtain a collection of resources, each of which represents an application. When you run a GET operation on this resource, you obtain a collection of Application Objects. See also [Application API](#Application_API) for syntax of the application object.
+
+### URI
+
+      * http://<nm http address:port>/ws/v1/node/apps
+
+### HTTP Operations Supported
+
+      * GET
+
+### Query Parameters Supported
+
+Multiple paramters can be specified.
+
+      * state - application state 
+      * user - user name
+
+### Elements of the *apps* (Applications) object
+
+When you make a request for the list of applications, the information will be returned as a collection of app objects. See also [Application API](#Application_API) for syntax of the app object.
+
+| Properties | Data Type | Description |
+|:---- |:---- |:---- |
+| app | array of app objects(JSON)/zero or more app objects(XML) | A collection of application objects |
+
+### Response Examples
+
+**JSON response**
+
+HTTP Request:
+
+      GET http://<nm http address:port>/ws/v1/node/apps
+
+Response Header:
+
+      HTTP/1.1 200 OK
+      Content-Type: application/json
+      Transfer-Encoding: chunked
+      Server: Jetty(6.1.26)
+
+Response Body:
+
+```json
+{
+   "apps" : {
+      "app" : [
+         {
+            "containerids" : [
+               "container_1326121700862_0003_01_000001",
+               "container_1326121700862_0003_01_000002"
+            ],
+            "user" : "user1",
+            "id" : "application_1326121700862_0003",
+            "state" : "RUNNING"
+         },
+         {
+            "user" : "user1",
+            "id" : "application_1326121700862_0002",
+            "state" : "FINISHED"
+         }
+      ]
+   }
+}
+```
+
+**XML response**
+
+HTTP Request:
+
+      GET http://<nm http address:port>/ws/v1/node/apps
+      Accept: application/xml
+
+Response Header:
+
+      HTTP/1.1 200 OK
+      Content-Type: application/xml
+      Content-Length: 400
+      Server: Jetty(6.1.26)
+
+Response Body:
+
+```xml
+<?xml version="1.0" encoding="UTF-8" standalone="yes"?>
+<apps>
+  <app>
+    <id>application_1326121700862_0002</id>
+    <state>FINISHED</state>
+    <user>user1</user>
+  </app>
+  <app>
+    <id>application_1326121700862_0003</id>
+    <state>RUNNING</state>
+    <user>user1</user>
+    <containerids>container_1326121700862_0003_01_000002</containerids>
+    <containerids>container_1326121700862_0003_01_000001</containerids>
+  </app>
+</apps>
+```
+
+Application API
+---------------
+
+An application resource contains information about a particular application that was run or is running on this NodeManager.
+
+### URI
+
+Use the following URI to obtain an app Object, for a application identified by the appid value.
+
+      * http://<nm http address:port>/ws/v1/node/apps/{appid}
+
+### HTTP Operations Supported
+
+      * GET
+
+### Query Parameters Supported
+
+      None
+
+### Elements of the *app* (Application) object
+
+| Properties | Data Type | Description |
+|:---- |:---- |:---- |
+| id | string | The application id |
+| user | string | The user who started the application |
+| state | string | The state of the application - valid states are: NEW, INITING, RUNNING, FINISHING\_CONTAINERS\_WAIT, APPLICATION\_RESOURCES\_CLEANINGUP, FINISHED |
+| containerids | array of containerids(JSON)/zero or more containerids(XML) | The list of containerids currently being used by the application on this node. If not present then no containers are currently running for this application. |
+
+### Response Examples
+
+**JSON response**
+
+HTTP Request:
+
+      GET http://<nm http address:port>/ws/v1/node/apps/application_1326121700862_0005
+
+Response Header:
+
+      HTTP/1.1 200 OK
+      Content-Type: application/json
+      Transfer-Encoding: chunked
+      Server: Jetty(6.1.26)
+
+Response Body:
+
+```json
+{
+   "app" : {
+      "containerids" : [
+         "container_1326121700862_0005_01_000003",
+         "container_1326121700862_0005_01_000001"
+      ],
+      "user" : "user1",
+      "id" : "application_1326121700862_0005",
+      "state" : "RUNNING"
+   }
+}
+```
+
+**XML response**
+
+HTTP Request:
+
+      GET http://<nm http address:port>/ws/v1/node/apps/application_1326121700862_0005
+      Accept: application/xml
+
+Response Header:
+
+      HTTP/1.1 200 OK
+      Content-Type: application/xml
+      Content-Length: 281 
+      Server: Jetty(6.1.26)
+
+Response Body:
+
+```xml
+<?xml version="1.0" encoding="UTF-8" standalone="yes"?>
+<app>
+  <id>application_1326121700862_0005</id>
+  <state>RUNNING</state>
+  <user>user1</user>
+  <containerids>container_1326121700862_0005_01_000003</containerids>
+  <containerids>container_1326121700862_0005_01_000001</containerids>
+</app>
+```
+
+Containers API
+--------------
+
+With the containers API, you can obtain a collection of resources, each of which represents a container. When you run a GET operation on this resource, you obtain a collection of Container Objects. See also [Container API](#Container_API) for syntax of the container object.
+
+### URI
+
+      * http://<nm http address:port>/ws/v1/node/containers
+
+### HTTP Operations Supported
+
+      * GET
+
+### Query Parameters Supported
+
+      None
+
+### Elements of the *containers* object
+
+When you make a request for the list of containers, the information will be returned as collection of container objects. See also [Container API](#Container_API) for syntax of the container object.
+
+| Properties | Data Type | Description |
+|:---- |:---- |:---- |
+| containers | array of container objects(JSON)/zero or more container objects(XML) | A collection of container objects |
+
+### Response Examples
+
+**JSON response**
+
+HTTP Request:
+
+      GET http://<nm http address:port>/ws/v1/node/containers
+
+Response Header:
+
+      HTTP/1.1 200 OK
+      Content-Type: application/json
+      Transfer-Encoding: chunked
+      Server: Jetty(6.1.26)
+
+Response Body:
+
+```json
+{
+   "containers" : {
+      "container" : [
+         {
+            "nodeId" : "host.domain.com:8041",
+            "totalMemoryNeededMB" : 2048,
+            "totalVCoresNeeded" : 1,
+            "state" : "RUNNING",
+            "diagnostics" : "",
+            "containerLogsLink" : "http://host.domain.com:8042/node/containerlogs/container_1326121700862_0006_01_000001/user1",
+            "user" : "user1",
+            "id" : "container_1326121700862_0006_01_000001",
+            "exitCode" : -1000
+         },
+         {
+            "nodeId" : "host.domain.com:8041",
+            "totalMemoryNeededMB" : 2048,
+            "totalVCoresNeeded" : 2,
+            "state" : "RUNNING",
+            "diagnostics" : "",
+            "containerLogsLink" : "http://host.domain.com:8042/node/containerlogs/container_1326121700862_0006_01_000003/user1",
+            "user" : "user1",
+            "id" : "container_1326121700862_0006_01_000003",
+            "exitCode" : -1000
+         }
+      ]
+   }
+}
+```
+
+**XML response**
+
+HTTP Request:
+
+      GET http://<nm http address:port>/ws/v1/node/containers
+      Accept: application/xml
+
+Response Header:
+
+      HTTP/1.1 200 OK
+      Content-Type: application/xml
+      Content-Length: 988
+      Server: Jetty(6.1.26)
+
+Response Body:
+
+```xml
+<?xml version="1.0" encoding="UTF-8" standalone="yes"?>
+<containers>
+  <container>
+    <id>container_1326121700862_0006_01_000001</id>
+    <state>RUNNING</state>
+    <exitCode>-1000</exitCode>
+    <diagnostics/>
+    <user>user1</user>
+    <totalMemoryNeededMB>2048</totalMemoryNeededMB>
+    <totalVCoresNeeded>1</totalVCoresNeeded>
+    <containerLogsLink>http://host.domain.com:8042/node/containerlogs/container_1326121700862_0006_01_000001/user1</containerLogsLink>
+    <nodeId>host.domain.com:8041</nodeId>
+  </container>
+  <container>
+    <id>container_1326121700862_0006_01_000003</id>
+    <state>DONE</state>
+    <exitCode>0</exitCode>
+    <diagnostics>Container killed by the ApplicationMaster.</diagnostics>
+    <user>user1</user>
+    <totalMemoryNeededMB>2048</totalMemoryNeededMB>
+    <totalVCoresNeeded>2</totalVCoresNeeded>
+    <containerLogsLink>http://host.domain.com:8042/node/containerlogs/container_1326121700862_0006_01_000003/user1</containerLogsLink>
+    <nodeId>host.domain.com:8041</nodeId>
+  </container>
+</containers>
+```
+
+Container API
+-------------
+
+A container resource contains information about a particular container that is running on this NodeManager.
+
+### URI
+
+Use the following URI to obtain a Container Object, from a container identified by the containerid value.
+
+      * http://<nm http address:port>/ws/v1/node/containers/{containerid}
+
+### HTTP Operations Supported
+
+      * GET
+
+### Query Parameters Supported
+
+      None
+
+### Elements of the *container* object
+
+| Properties | Data Type | Description |
+|:---- |:---- |:---- |
+| id | string | The container id |
+| state | string | State of the container - valid states are: NEW, LOCALIZING, LOCALIZATION\_FAILED, LOCALIZED, RUNNING, EXITED\_WITH\_SUCCESS, EXITED\_WITH\_FAILURE, KILLING, CONTAINER\_CLEANEDUP\_AFTER\_KILL, CONTAINER\_RESOURCES\_CLEANINGUP, DONE |
+| nodeId | string | The id of the node the container is on |
+| containerLogsLink | string | The http link to the container logs |
+| user | string | The user name of the user which started the container |
+| exitCode | int | Exit code of the container |
+| diagnostics | string | A diagnostic message for failed containers |
+| totalMemoryNeededMB | long | Total amout of memory needed by the container (in MB) |
+| totalVCoresNeeded | long | Total number of virtual cores needed by the container |
+
+### Response Examples
+
+**JSON response**
+
+HTTP Request:
+
+      GET http://<nm http address:port>/ws/v1/nodes/containers/container_1326121700862_0007_01_000001
+
+Response Header:
+
+      HTTP/1.1 200 OK
+      Content-Type: application/json
+      Transfer-Encoding: chunked
+      Server: Jetty(6.1.26)
+
+Response Body:
+
+```json
+{
+   "container" : {
+      "nodeId" : "host.domain.com:8041",
+      "totalMemoryNeededMB" : 2048,
+      "totalVCoresNeeded" : 1,
+      "state" : "RUNNING",
+      "diagnostics" : "",
+      "containerLogsLink" : "http://host.domain.com:8042/node/containerlogs/container_1326121700862_0007_01_000001/user1",
+      "user" : "user1",
+      "id" : "container_1326121700862_0007_01_000001",
+      "exitCode" : -1000
+   }
+}
+```
+
+**XML response**
+
+HTTP Request:
+
+      GET http://<nm http address:port>/ws/v1/node/containers/container_1326121700862_0007_01_000001
+      Accept: application/xml
+
+Response Header:
+
+      HTTP/1.1 200 OK
+      Content-Type: application/xml
+      Content-Length: 491 
+      Server: Jetty(6.1.26)
+
+Response Body:
+
+```xml
+<?xml version="1.0" encoding="UTF-8" standalone="yes"?>
+<container>
+  <id>container_1326121700862_0007_01_000001</id>
+  <state>RUNNING</state>
+  <exitCode>-1000</exitCode>
+  <diagnostics/>
+  <user>user1</user>
+  <totalMemoryNeededMB>2048</totalMemoryNeededMB>
+  <totalVCoresNeeded>1</totalVCoresNeeded>
+  <containerLogsLink>http://host.domain.com:8042/node/containerlogs/container_1326121700862_0007_01_000001/user1</containerLogsLink>
+  <nodeId>host.domain.com:8041</nodeId>
+</container>
+```
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/06aca7c6/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/NodeManagerRestart.md
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/NodeManagerRestart.md b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/NodeManagerRestart.md
new file mode 100644
index 0000000..be7d75b
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/NodeManagerRestart.md
@@ -0,0 +1,53 @@
+<!---
+  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.
+-->
+
+NodeManager Restart
+===================
+
+* [Introduction](#Introduction)
+* [Enabling NM Restart](#Enabling_NM_Restart)
+
+Introduction
+------------
+
+This document gives an overview of NodeManager (NM) restart, a feature that enables the NodeManager to be restarted without losing the active containers running on the node. At a high level, the NM stores any necessary state to a local state-store as it processes container-management requests. When the NM restarts, it recovers by first loading state for various subsystems and then letting those subsystems perform recovery using the loaded state.
+
+Enabling NM Restart
+-------------------
+
+Step 1. To enable NM Restart functionality, set the following property in **conf/yarn-site.xml** to *true*.
+
+| Property | Value |
+|:---- |:---- |
+| `yarn.nodemanager.recovery.enabled` | `true`, (default value is set to false) |
+
+Step 2.  Configure a path to the local file-system directory where the NodeManager can save its run state.
+
+| Property | Description |
+|:---- |:---- |
+| `yarn.nodemanager.recovery.dir` | The local filesystem directory in which the node manager will store state when recovery is enabled. The default value is set to `$hadoop.tmp.dir/yarn-nm-recovery`. |
+
+Step 3.  Configure a valid RPC address for the NodeManager.
+
+| Property | Description |
+|:---- |:---- |
+| `yarn.nodemanager.address` | Ephemeral ports (port 0, which is default) cannot be used for the NodeManager's RPC server specified via yarn.nodemanager.address as it can make NM use different ports before and after a restart. This will break any previously running clients that were communicating with the NM before restart. Explicitly setting yarn.nodemanager.address to an address with specific port number (for e.g 0.0.0.0:45454) is a precondition for enabling NM restart. |
+
+Step 4.  Auxiliary services.
+
+  * NodeManagers in a YARN cluster can be configured to run auxiliary services. For a completely functional NM restart, YARN relies on any auxiliary service configured to also support recovery. This usually includes (1) avoiding usage of ephemeral ports so that previously running clients (in this case, usually containers) are not disrupted after restart and (2) having the auxiliary service itself support recoverability by reloading any previous state when NodeManager restarts and reinitializes the auxiliary service.
+
+  * A simple example for the above is the auxiliary service 'ShuffleHandler' for MapReduce (MR). ShuffleHandler respects the above two requirements already, so users/admins don't have do anything for it to support NM restart: (1) The configuration property **mapreduce.shuffle.port** controls which port the ShuffleHandler on a NodeManager host binds to, and it defaults to a non-ephemeral port. (2) The ShuffleHandler service also already supports recovery of previous state after NM restarts.
+
+

http://git-wip-us.apache.org/repos/asf/hadoop/blob/06aca7c6/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/ResourceManagerHA.md
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/ResourceManagerHA.md b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/ResourceManagerHA.md
new file mode 100644
index 0000000..491b885
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/ResourceManagerHA.md
@@ -0,0 +1,140 @@
+<!---
+  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.
+-->
+
+ResourceManager High Availability
+=================================
+
+* [Introduction](#Introduction)
+* [Architecture](#Architecture)
+    * [RM Failover](#RM_Failover)
+    * [Recovering prevous active-RM's state](#Recovering_prevous_active-RMs_state)
+* [Deployment](#Deployment)
+    * [Configurations](#Configurations)
+    * [Admin commands](#Admin_commands)
+    * [ResourceManager Web UI services](#ResourceManager_Web_UI_services)
+    * [Web Services](#Web_Services)
+
+Introduction
+------------
+
+This guide provides an overview of High Availability of YARN's ResourceManager, and details how to configure and use this feature. The ResourceManager (RM) is responsible for tracking the resources in a cluster, and scheduling applications (e.g., MapReduce jobs). Prior to Hadoop 2.4, the ResourceManager is the single point of failure in a YARN cluster. The High Availability feature adds redundancy in the form of an Active/Standby ResourceManager pair to remove this otherwise single point of failure.
+
+Architecture
+------------
+
+![Overview of ResourceManager High Availability](images/rm-ha-overview.png)
+
+### RM Failover
+
+ResourceManager HA is realized through an Active/Standby architecture - at any point of time, one of the RMs is Active, and one or more RMs are in Standby mode waiting to take over should anything happen to the Active. The trigger to transition-to-active comes from either the admin (through CLI) or through the integrated failover-controller when automatic-failover is enabled.
+
+#### Manual transitions and failover
+
+When automatic failover is not enabled, admins have to manually transition one of the RMs to Active. To failover from one RM to the other, they are expected to first transition the Active-RM to Standby and transition a Standby-RM to Active. All this can be done using the "`yarn rmadmin`" CLI.
+
+#### Automatic failover
+
+The RMs have an option to embed the Zookeeper-based ActiveStandbyElector to decide which RM should be the Active. When the Active goes down or becomes unresponsive, another RM is automatically elected to be the Active which then takes over. Note that, there is no need to run a separate ZKFC daemon as is the case for HDFS because ActiveStandbyElector embedded in RMs acts as a failure detector and a leader elector instead of a separate ZKFC deamon.
+
+#### Client, ApplicationMaster and NodeManager on RM failover
+
+When there are multiple RMs, the configuration (yarn-site.xml) used by clients and nodes is expected to list all the RMs. Clients, ApplicationMasters (AMs) and NodeManagers (NMs) try connecting to the RMs in a round-robin fashion until they hit the Active RM. If the Active goes down, they resume the round-robin polling until they hit the "new" Active. This default retry logic is implemented as `org.apache.hadoop.yarn.client.ConfiguredRMFailoverProxyProvider`. You can override the logic by implementing `org.apache.hadoop.yarn.client.RMFailoverProxyProvider` and setting the value of `yarn.client.failover-proxy-provider` to the class name.
+
+### Recovering prevous active-RM's state
+
+With the [ResourceManger Restart](./ResourceManagerRestart.html) enabled, the RM being promoted to an active state loads the RM internal state and continues to operate from where the previous active left off as much as possible depending on the RM restart feature. A new attempt is spawned for each managed application previously submitted to the RM. Applications can checkpoint periodically to avoid losing any work. The state-store must be visible from the both of Active/Standby RMs. Currently, there are two RMStateStore implementations for persistence - FileSystemRMStateStore and ZKRMStateStore. The `ZKRMStateStore` implicitly allows write access to a single RM at any point in time, and hence is the recommended store to use in an HA cluster. When using the ZKRMStateStore, there is no need for a separate fencing mechanism to address a potential split-brain situation where multiple RMs can potentially assume the Active role.
+
+Deployment
+----------
+
+### Configurations
+
+Most of the failover functionality is tunable using various configuration properties. Following is a list of required/important ones. yarn-default.xml carries a full-list of knobs. See [yarn-default.xml](../hadoop-yarn-common/yarn-default.xml) for more information including default values. See the document for [ResourceManger Restart](./ResourceManagerRestart.html) also for instructions on setting up the state-store.
+
+| Configuration Properties | Description |
+|:---- |:---- |
+| `yarn.resourcemanager.zk-address` | Address of the ZK-quorum. Used both for the state-store and embedded leader-election. |
+| `yarn.resourcemanager.ha.enabled` | Enable RM HA. |
+| `yarn.resourcemanager.ha.rm-ids` | List of logical IDs for the RMs. e.g., "rm1,rm2". |
+| `yarn.resourcemanager.hostname.*rm-id*` | For each *rm-id*, specify the hostname the RM corresponds to. Alternately, one could set each of the RM's service addresses. |
+| `yarn.resourcemanager.ha.id` | Identifies the RM in the ensemble. This is optional; however, if set, admins have to ensure that all the RMs have their own IDs in the config. |
+| `yarn.resourcemanager.ha.automatic-failover.enabled` | Enable automatic failover; By default, it is enabled only when HA is enabled. |
+| `yarn.resourcemanager.ha.automatic-failover.embedded` | Use embedded leader-elector to pick the Active RM, when automatic failover is enabled. By default, it is enabled only when HA is enabled. |
+| `yarn.resourcemanager.cluster-id` | Identifies the cluster. Used by the elector to ensure an RM doesn't take over as Active for another cluster. |
+| `yarn.client.failover-proxy-provider` | The class to be used by Clients, AMs and NMs to failover to the Active RM. |
+| `yarn.client.failover-max-attempts` | The max number of times FailoverProxyProvider should attempt failover. |
+| `yarn.client.failover-sleep-base-ms` | The sleep base (in milliseconds) to be used for calculating the exponential delay between failovers. |
+| `yarn.client.failover-sleep-max-ms` | The maximum sleep time (in milliseconds) between failovers. |
+| `yarn.client.failover-retries` | The number of retries per attempt to connect to a ResourceManager. |
+| `yarn.client.failover-retries-on-socket-timeouts` | The number of retries per attempt to connect to a ResourceManager on socket timeouts. |
+
+#### Sample configurations
+
+Here is the sample of minimal setup for RM failover.
+
+```xml
+<property>
+  <name>yarn.resourcemanager.ha.enabled</name>
+  <value>true</value>
+</property>
+<property>
+  <name>yarn.resourcemanager.cluster-id</name>
+  <value>cluster1</value>
+</property>
+<property>
+  <name>yarn.resourcemanager.ha.rm-ids</name>
+  <value>rm1,rm2</value>
+</property>
+<property>
+  <name>yarn.resourcemanager.hostname.rm1</name>
+  <value>master1</value>
+</property>
+<property>
+  <name>yarn.resourcemanager.hostname.rm2</name>
+  <value>master2</value>
+</property>
+<property>
+  <name>yarn.resourcemanager.zk-address</name>
+  <value>zk1:2181,zk2:2181,zk3:2181</value>
+</property>
+```
+
+### Admin commands
+
+`yarn rmadmin` has a few HA-specific command options to check the health/state of an RM, and transition to Active/Standby. Commands for HA take service id of RM set by `yarn.resourcemanager.ha.rm-ids` as argument.
+
+     $ yarn rmadmin -getServiceState rm1
+     active
+     
+     $ yarn rmadmin -getServiceState rm2
+     standby
+
+If automatic failover is enabled, you can not use manual transition command. Though you can override this by --forcemanual flag, you need caution.
+
+     $ yarn rmadmin -transitionToStandby rm1
+     Automatic failover is enabled for org.apache.hadoop.yarn.client.RMHAServiceTarget@1d8299fd
+     Refusing to manually manage HA state, since it may cause
+     a split-brain scenario or other incorrect state.
+     If you are very sure you know what you are doing, please
+     specify the forcemanual flag.
+
+See [YarnCommands](./YarnCommands.html) for more details.
+
+### ResourceManager Web UI services
+
+Assuming a standby RM is up and running, the Standby automatically redirects all web requests to the Active, except for the "About" page.
+
+### Web Services
+
+Assuming a standby RM is up and running, RM web-services described at [ResourceManager REST APIs](./ResourceManagerRest.html) when invoked on a standby RM are automatically redirected to the Active RM.


[11/50] [abbrv] hadoop git commit: HDFS-7537. Add "UNDER MIN REPL'D BLOCKS" count to fsck. Contributed by GAO Rui

Posted by zh...@apache.org.
HDFS-7537. Add "UNDER MIN REPL'D BLOCKS" count to fsck.  Contributed by GAO Rui


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

Branch: refs/heads/HDFS-7285
Commit: 0127820a0dc6f0d4f741e276a1048117f0e78fc5
Parents: 21e9e91
Author: Tsz-Wo Nicholas Sze <sz...@hortonworks.com>
Authored: Thu Feb 26 11:45:56 2015 +0800
Committer: Zhe Zhang <zh...@cloudera.com>
Committed: Mon Mar 2 09:13:51 2015 -0800

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |  3 +
 .../hdfs/server/namenode/NamenodeFsck.java      | 36 +++++++--
 .../hadoop/hdfs/server/namenode/TestFsck.java   | 81 +++++++++++++++++++-
 3 files changed, 111 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/0127820a/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 6dc7a0f..4523bf4 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -673,6 +673,9 @@ Release 2.7.0 - UNRELEASED
     HDFS-7495. Remove updatePosition argument from DFSInputStream#getBlockAt()
     (cmccabe)
 
+    HDFS-7537. Add "UNDER MIN REPL'D BLOCKS" count to fsck.  (GAO Rui via
+    szetszwo)
+
   OPTIMIZATIONS
 
     HDFS-7454. Reduce memory footprint for AclEntries in NameNode.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0127820a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java
index f36b773..3c7918f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java
@@ -513,6 +513,9 @@ public class NamenodeFsck implements DataEncryptionKeyFactory {
       res.totalReplicas += liveReplicas;
       short targetFileReplication = file.getReplication();
       res.numExpectedReplicas += targetFileReplication;
+      if(liveReplicas<minReplication){
+        res.numUnderMinReplicatedBlocks++;
+      }
       if (liveReplicas > targetFileReplication) {
         res.excessiveReplicas += (liveReplicas - targetFileReplication);
         res.numOverReplicatedBlocks += 1;
@@ -859,6 +862,7 @@ public class NamenodeFsck implements DataEncryptionKeyFactory {
     long corruptBlocks = 0L;
     long excessiveReplicas = 0L;
     long missingReplicas = 0L;
+    long numUnderMinReplicatedBlocks=0L;
     long numOverReplicatedBlocks = 0L;
     long numUnderReplicatedBlocks = 0L;
     long numMisReplicatedBlocks = 0L;  // blocks that do not satisfy block placement policy
@@ -875,10 +879,13 @@ public class NamenodeFsck implements DataEncryptionKeyFactory {
     long totalReplicas = 0L;
 
     final short replication;
+    final int minReplication;
     
     Result(Configuration conf) {
       this.replication = (short)conf.getInt(DFSConfigKeys.DFS_REPLICATION_KEY, 
                                             DFSConfigKeys.DFS_REPLICATION_DEFAULT);
+      this.minReplication = (short)conf.getInt(DFSConfigKeys.DFS_NAMENODE_REPLICATION_MIN_KEY,
+                                            DFSConfigKeys.DFS_NAMENODE_REPLICATION_MIN_DEFAULT);
     }
     
     /**
@@ -926,15 +933,28 @@ public class NamenodeFsck implements DataEncryptionKeyFactory {
         res.append(" (Total open file blocks (not validated): ").append(
             totalOpenFilesBlocks).append(")");
       }
-      if (corruptFiles > 0) {
-        res.append("\n  ********************************").append(
-            "\n  CORRUPT FILES:\t").append(corruptFiles);
-        if (missingSize > 0) {
-          res.append("\n  MISSING BLOCKS:\t").append(missingIds.size()).append(
-              "\n  MISSING SIZE:\t\t").append(missingSize).append(" B");
+      if (corruptFiles > 0 || numUnderMinReplicatedBlocks>0) {
+        res.append("\n  ********************************");
+        if(numUnderMinReplicatedBlocks>0){
+          res.append("\n  UNDER MIN REPL'D BLOCKS:\t").append(numUnderMinReplicatedBlocks);
+          if(totalBlocks>0){
+            res.append(" (").append(
+                ((float) (numUnderMinReplicatedBlocks * 100) / (float) totalBlocks))
+                .append(" %)");
+          }
+          res.append("\n  ").append("DFSConfigKeys.DFS_NAMENODE_REPLICATION_MIN_KEY:\t")
+             .append(minReplication);
         }
-        if (corruptBlocks > 0) {
-          res.append("\n  CORRUPT BLOCKS: \t").append(corruptBlocks);
+        if(corruptFiles>0) {
+          res.append(
+              "\n  CORRUPT FILES:\t").append(corruptFiles);
+          if (missingSize > 0) {
+            res.append("\n  MISSING BLOCKS:\t").append(missingIds.size()).append(
+                "\n  MISSING SIZE:\t\t").append(missingSize).append(" B");
+          }
+          if (corruptBlocks > 0) {
+            res.append("\n  CORRUPT BLOCKS: \t").append(corruptBlocks);
+          }
         }
         res.append("\n  ********************************");
       }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0127820a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFsck.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFsck.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFsck.java
index 7cdf5ec..33de692 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFsck.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFsck.java
@@ -693,7 +693,86 @@ public class TestFsck {
       if (cluster != null) {cluster.shutdown();}
     }
   }
-  
+
+  @Test
+  public void testUnderMinReplicatedBlock() throws Exception {
+    Configuration conf = new HdfsConfiguration();
+    conf.setLong(DFSConfigKeys.DFS_BLOCKREPORT_INTERVAL_MSEC_KEY, 1000);
+    // Set short retry timeouts so this test runs faster
+    conf.setInt(DFSConfigKeys.DFS_CLIENT_RETRY_WINDOW_BASE, 10);
+    // Set minReplication to 2
+    short minReplication=2;
+    conf.setInt(DFSConfigKeys.DFS_NAMENODE_REPLICATION_MIN_KEY,minReplication);
+    FileSystem fs = null;
+    DFSClient dfsClient = null;
+    LocatedBlocks blocks = null;
+    int replicaCount = 0;
+    Random random = new Random();
+    String outStr = null;
+    short factor = 1;
+    MiniDFSCluster cluster = null;
+    try {
+      cluster = new MiniDFSCluster.Builder(conf).numDataNodes(2).build();
+      cluster.waitActive();
+      fs = cluster.getFileSystem();
+      Path file1 = new Path("/testUnderMinReplicatedBlock");
+      DFSTestUtil.createFile(fs, file1, 1024, minReplication, 0);
+      // Wait until file replication has completed
+      DFSTestUtil.waitReplication(fs, file1, minReplication);
+      ExtendedBlock block = DFSTestUtil.getFirstBlock(fs, file1);
+
+      // Make sure filesystem is in healthy state
+      outStr = runFsck(conf, 0, true, "/");
+      System.out.println(outStr);
+      assertTrue(outStr.contains(NamenodeFsck.HEALTHY_STATUS));
+
+      // corrupt the first replica
+      File blockFile = cluster.getBlockFile(0, block);
+      if (blockFile != null && blockFile.exists()) {
+        RandomAccessFile raFile = new RandomAccessFile(blockFile, "rw");
+        FileChannel channel = raFile.getChannel();
+        String badString = "BADBAD";
+        int rand = random.nextInt((int) channel.size()/2);
+        raFile.seek(rand);
+        raFile.write(badString.getBytes());
+        raFile.close();
+      }
+
+      dfsClient = new DFSClient(new InetSocketAddress("localhost",
+          cluster.getNameNodePort()), conf);
+      blocks = dfsClient.getNamenode().
+          getBlockLocations(file1.toString(), 0, Long.MAX_VALUE);
+      replicaCount = blocks.get(0).getLocations().length;
+      while (replicaCount != factor) {
+        try {
+          Thread.sleep(100);
+          // Read the file to trigger reportBadBlocks
+          try {
+            IOUtils.copyBytes(fs.open(file1), new IOUtils.NullOutputStream(), conf,
+                true);
+          } catch (IOException ie) {
+            // Ignore exception
+          }
+          System.out.println("sleep in try: replicaCount="+replicaCount+"  factor="+factor);
+        } catch (InterruptedException ignore) {
+        }
+        blocks = dfsClient.getNamenode().
+            getBlockLocations(file1.toString(), 0, Long.MAX_VALUE);
+        replicaCount = blocks.get(0).getLocations().length;
+      }
+
+      // Check if fsck reports the same
+      outStr = runFsck(conf, 0, true, "/");
+      System.out.println(outStr);
+      assertTrue(outStr.contains(NamenodeFsck.HEALTHY_STATUS));
+      assertTrue(outStr.contains("UNDER MIN REPL'D BLOCKS:\t1 (100.0 %)"));
+      assertTrue(outStr.contains("DFSConfigKeys.DFS_NAMENODE_REPLICATION_MIN_KEY:\t2"));
+    } finally {
+      if (cluster != null) {cluster.shutdown();}
+    }
+  }
+
+
   /** Test if fsck can return -1 in case of failure
    * 
    * @throws Exception


[08/50] [abbrv] hadoop git commit: HDFS-7832. Show 'Last Modified' in Namenode's 'Browse Filesystem' (Contributed by Vinayakumar B)

Posted by zh...@apache.org.
HDFS-7832. Show 'Last Modified' in Namenode's 'Browse Filesystem' (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/199a82d4
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/199a82d4
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/199a82d4

Branch: refs/heads/HDFS-7285
Commit: 199a82d4a68ebfacab4d2e1b263d332b8f2156dc
Parents: 9f356cf
Author: Vinayakumar B <vi...@apache.org>
Authored: Thu Feb 26 14:36:09 2015 +0530
Committer: Zhe Zhang <zh...@cloudera.com>
Committed: Mon Mar 2 09:13:51 2015 -0800

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt                 | 3 +++
 .../hadoop-hdfs/src/main/webapps/hdfs/explorer.html         | 2 ++
 .../hadoop-hdfs/src/main/webapps/hdfs/explorer.js           | 9 ++++++++-
 3 files changed, 13 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/199a82d4/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 4523bf4..e09714f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -676,6 +676,9 @@ Release 2.7.0 - UNRELEASED
     HDFS-7537. Add "UNDER MIN REPL'D BLOCKS" count to fsck.  (GAO Rui via
     szetszwo)
 
+    HDFS-7832. Show 'Last Modified' in Namenode's 'Browse Filesystem'
+    (vinayakumarb)
+
   OPTIMIZATIONS
 
     HDFS-7454. Reduce memory footprint for AclEntries in NameNode.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/199a82d4/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/explorer.html
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/explorer.html b/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/explorer.html
index e1fdfa3..7b34044 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/explorer.html
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/explorer.html
@@ -112,6 +112,7 @@
             <th>Owner</th>
             <th>Group</th>
             <th>Size</th>
+            <th>Last Modified</th>
             <th>Replication</th>
             <th>Block Size</th>
             <th>Name</th>
@@ -124,6 +125,7 @@
             <td>{owner}</td>
             <td>{group}</td>
             <td>{length|fmt_bytes}</td>
+            <td>{#helper_date_tostring value="{modificationTime}"/}</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>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/199a82d4/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/explorer.js
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/explorer.js b/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/explorer.js
index 87d47fa..0a53dcd 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/explorer.js
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/explorer.js
@@ -143,6 +143,12 @@
   }
 
   function browse_directory(dir) {
+    var HELPERS = {
+      'helper_date_tostring' : function (chunk, ctx, bodies, params) {
+        var value = dust.helpers.tap(params.value, chunk, ctx);
+        return chunk.write('' + new Date(Number(value)).toLocaleString());
+      }
+    };
     var url = '/webhdfs/v1' + dir + '?op=LISTSTATUS';
     $.get(url, function(data) {
       var d = get_response(data, "FileStatuses");
@@ -154,7 +160,8 @@
       current_directory = dir;
       $('#directory').val(dir);
       window.location.hash = dir;
-      dust.render('explorer', d, function(err, out) {
+      var base = dust.makeBase(HELPERS);
+      dust.render('explorer', base.push(d), function(err, out) {
         $('#panel').html(out);
 
         $('.explorer-browse-links').click(function() {


[41/50] [abbrv] hadoop git commit: HDFS-7685. Document dfs.namenode.heartbeat.recheck-interval in hdfs-default.xml. Contributed by Kai Sasaki.

Posted by zh...@apache.org.
HDFS-7685. Document dfs.namenode.heartbeat.recheck-interval in hdfs-default.xml. Contributed by Kai Sasaki.


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

Branch: refs/heads/HDFS-7285
Commit: 70c1fa0b18639c2aab57efdbd427901c95f4a728
Parents: 4e95f98
Author: Akira Ajisaka <aa...@apache.org>
Authored: Fri Feb 27 12:17:34 2015 -0800
Committer: Zhe Zhang <zh...@cloudera.com>
Committed: Mon Mar 2 09:13:54 2015 -0800

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt              |  3 +++
 .../hadoop-hdfs/src/main/resources/hdfs-default.xml      | 11 +++++++++++
 2 files changed, 14 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/70c1fa0b/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 b2422d6..b4b0087 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -685,6 +685,9 @@ Release 2.7.0 - UNRELEASED
     HDFS-7308. Change the packet chunk size computation in DFSOutputStream in
     order to enforce packet size <= 64kB.  (Takuya Fukudome via szetszwo)
 
+    HDFS-7685. Document dfs.namenode.heartbeat.recheck-interval in
+    hdfs-default.xml. (Kai Sasaki via aajisaka)
+
   OPTIMIZATIONS
 
     HDFS-7454. Reduce memory footprint for AclEntries in NameNode.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/70c1fa0b/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml b/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
index 85d2273..66fe86c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
@@ -145,6 +145,17 @@
 </property>
 
 <property>
+  <name>dfs.namenode.heartbeat.recheck-interval</name>
+  <value>300000</value>
+  <description>
+    This time decides the interval to check for expired datanodes.
+    With this value and dfs.heartbeat.interval, the interval of
+    deciding the datanode is stale or not is also calculated.
+    The unit of this configuration is millisecond.
+  </description>
+</property>
+
+<property>
   <name>dfs.http.policy</name>
   <value>HTTP_ONLY</value>
   <description>Decide if HTTPS(SSL) is supported on HDFS


[39/50] [abbrv] hadoop git commit: move HADOOP-10976 to 2.7

Posted by zh...@apache.org.
move HADOOP-10976 to 2.7


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

Branch: refs/heads/HDFS-7285
Commit: 779724926e7e4b08c89b79684065b66623dd5a97
Parents: e70ce6f
Author: Akira Ajisaka <aa...@apache.org>
Authored: Sat Feb 28 17:15:13 2015 -0800
Committer: Zhe Zhang <zh...@cloudera.com>
Committed: Mon Mar 2 09:13:54 2015 -0800

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


http://git-wip-us.apache.org/repos/asf/hadoop/blob/77972492/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 6d4da77..74bf558 100644
--- a/hadoop-common-project/hadoop-common/CHANGES.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES.txt
@@ -163,9 +163,6 @@ Trunk (Unreleased)
     HADOOP-11346. Rewrite sls/rumen to use new shell framework (John Smith
     via aw)
 
-    HADOOP-10976. moving the source code of hadoop-tools docs to the
-    directory under hadoop-tools (Masatake Iwasaki via aw)
-
     HADOOP-7713. dfs -count -q should label output column (Jonathan Allen
     via aw)
 
@@ -636,6 +633,9 @@ Release 2.7.0 - UNRELEASED
     HADOOP-11632. Cleanup Find.java to remove SupressWarnings annotations.
     (Akira Ajisaka via ozawa)
 
+    HADOOP-10976. moving the source code of hadoop-tools docs to the
+    directory under hadoop-tools (Masatake Iwasaki via aw)
+
   OPTIMIZATIONS
 
     HADOOP-11323. WritableComparator#compare keeps reference to byte array.