You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-commits@hadoop.apache.org by ae...@apache.org on 2016/06/19 00:32:40 UTC

[01/39] hadoop git commit: HADOOP-13270. BZip2CompressionInputStream finds the same compression marker twice in corner case, causing duplicate data blocks. Contributed by Kai Sasaki.

Repository: hadoop
Updated Branches:
  refs/heads/HDFS-1312 13599a8a4 -> 91d62aaf1


HADOOP-13270. BZip2CompressionInputStream finds the same compression marker twice in corner case, causing duplicate data blocks. 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/e3ba9ad3
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/e3ba9ad3
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/e3ba9ad3

Branch: refs/heads/HDFS-1312
Commit: e3ba9ad3f116306910f74645ded91506345b9f6e
Parents: 709a814
Author: Akira Ajisaka <aa...@apache.org>
Authored: Tue Jun 14 10:18:17 2016 +0900
Committer: Akira Ajisaka <aa...@apache.org>
Committed: Tue Jun 14 10:18:41 2016 +0900

----------------------------------------------------------------------
 .../apache/hadoop/io/compress/BZip2Codec.java   |   7 +-
 .../hadoop/mapred/TestTextInputFormat.java      | 108 ++++++++++---------
 2 files changed, 63 insertions(+), 52 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/e3ba9ad3/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/BZip2Codec.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/BZip2Codec.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/BZip2Codec.java
index 2c5a7be..49dd9c1 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/BZip2Codec.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/BZip2Codec.java
@@ -207,7 +207,12 @@ public class BZip2Codec implements Configurable, SplittableCompressionCodec {
     // time stream might start without a leading BZ.
     final long FIRST_BZIP2_BLOCK_MARKER_POSITION =
       CBZip2InputStream.numberOfBytesTillNextMarker(seekableIn);
-    long adjStart = Math.max(0L, start - FIRST_BZIP2_BLOCK_MARKER_POSITION);
+    long adjStart = 0L;
+    if (start != 0) {
+      // Other than the first of file, the marker size is 6 bytes.
+      adjStart = Math.max(0L, start - (FIRST_BZIP2_BLOCK_MARKER_POSITION
+          - (HEADER_LEN + SUB_HEADER_LEN)));
+    }
 
     ((Seekable)seekableIn).seek(adjStart);
     SplitCompressionInputStream in =

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e3ba9ad3/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestTextInputFormat.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestTextInputFormat.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestTextInputFormat.java
index b833b60..5106c38 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestTextInputFormat.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestTextInputFormat.java
@@ -175,69 +175,75 @@ public class TestTextInputFormat {
     for (int length = MAX_LENGTH / 2; length < MAX_LENGTH;
         length += random.nextInt(MAX_LENGTH / 4)+1) {
 
-      LOG.info("creating; entries = " + length);
-
-
-      // create a file with length entries
-      Writer writer =
-        new OutputStreamWriter(codec.createOutputStream(localFs.create(file)));
-      try {
-        for (int i = 0; i < length; i++) {
-          writer.write(Integer.toString(i));
-          writer.write("\n");
-        }
-      } finally {
-        writer.close();
+      for (int i = 0; i < 3; i++) {
+        int numSplits = random.nextInt(MAX_LENGTH / 2000) + 1;
+        verifyPartitions(length, numSplits, file, codec, conf);
       }
+    }
 
-      // try splitting the file in a variety of sizes
-      TextInputFormat format = new TextInputFormat();
-      format.configure(conf);
-      LongWritable key = new LongWritable();
-      Text value = new Text();
-      for (int i = 0; i < 3; i++) {
-        int numSplits = random.nextInt(MAX_LENGTH/2000)+1;
-        LOG.info("splitting: requesting = " + numSplits);
-        InputSplit[] splits = format.getSplits(conf, numSplits);
-        LOG.info("splitting: got =        " + splits.length);
+    // corner case when we have byte alignment and position of stream are same
+    verifyPartitions(471507, 218, file, codec, conf);
+    verifyPartitions(473608, 110, file, codec, conf);
+  }
 
+  private void verifyPartitions(int length, int numSplits, Path file,
+      CompressionCodec codec, JobConf conf) throws IOException {
 
+    LOG.info("creating; entries = " + length);
 
-        // check each split
-        BitSet bits = new BitSet(length);
-        for (int j = 0; j < splits.length; j++) {
-          LOG.debug("split["+j+"]= " + splits[j]);
-          RecordReader<LongWritable, Text> reader =
-            format.getRecordReader(splits[j], conf, reporter);
-          try {
-            int counter = 0;
-            while (reader.next(key, value)) {
-              int v = Integer.parseInt(value.toString());
-              LOG.debug("read " + v);
 
-              if (bits.get(v)) {
-                LOG.warn("conflict with " + v +
+    // create a file with length entries
+    Writer writer =
+        new OutputStreamWriter(codec.createOutputStream(localFs.create(file)));
+    try {
+      for (int i = 0; i < length; i++) {
+        writer.write(Integer.toString(i));
+        writer.write("\n");
+      }
+    } finally {
+      writer.close();
+    }
+
+    // try splitting the file in a variety of sizes
+    TextInputFormat format = new TextInputFormat();
+    format.configure(conf);
+    LongWritable key = new LongWritable();
+    Text value = new Text();
+    LOG.info("splitting: requesting = " + numSplits);
+    InputSplit[] splits = format.getSplits(conf, numSplits);
+    LOG.info("splitting: got =        " + splits.length);
+
+
+    // check each split
+    BitSet bits = new BitSet(length);
+    for (int j = 0; j < splits.length; j++) {
+      LOG.debug("split["+j+"]= " + splits[j]);
+      RecordReader<LongWritable, Text> reader =
+              format.getRecordReader(splits[j], conf, Reporter.NULL);
+      try {
+        int counter = 0;
+        while (reader.next(key, value)) {
+          int v = Integer.parseInt(value.toString());
+          LOG.debug("read " + v);
+          if (bits.get(v)) {
+            LOG.warn("conflict with " + v +
                     " in split " + j +
                     " at position "+reader.getPos());
-              }
-              assertFalse("Key in multiple partitions.", bits.get(v));
-              bits.set(v);
-              counter++;
-            }
-            if (counter > 0) {
-              LOG.info("splits["+j+"]="+splits[j]+" count=" + counter);
-            } else {
-              LOG.debug("splits["+j+"]="+splits[j]+" count=" + counter);
-            }
-          } finally {
-            reader.close();
           }
+          assertFalse("Key in multiple partitions.", bits.get(v));
+          bits.set(v);
+          counter++;
         }
-        assertEquals("Some keys in no partition.", length, bits.cardinality());
+        if (counter > 0) {
+          LOG.info("splits["+j+"]="+splits[j]+" count=" + counter);
+        } else {
+          LOG.debug("splits["+j+"]="+splits[j]+" count=" + counter);
+        }
+      } finally {
+        reader.close();
       }
-
     }
-
+    assertEquals("Some keys in no partition.", length, bits.cardinality());
   }
 
   private static LineReader makeStream(String str) throws IOException {


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


[17/39] hadoop git commit: HDFS-10505. OIV's ReverseXML processor should support ACLs (Surendra Singh Lilhore via cmccabe)

Posted by ae...@apache.org.
HDFS-10505. OIV's ReverseXML processor should support ACLs (Surendra Singh Lilhore via cmccabe)


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

Branch: refs/heads/HDFS-1312
Commit: 2449db507d84b1c4fac70a800fb2ad8905cf3db7
Parents: b48f27e
Author: Colin Patrick Mccabe <cm...@cloudera.com>
Authored: Wed Jun 15 22:35:19 2016 -0700
Committer: Colin Patrick Mccabe <cm...@cloudera.com>
Committed: Wed Jun 15 22:35:19 2016 -0700

----------------------------------------------------------------------
 .../server/namenode/FSImageFormatPBINode.java   | 10 +++----
 .../OfflineImageReconstructor.java              | 30 ++++++++++++++++++--
 .../TestOfflineImageViewer.java                 | 19 +++++++++++++
 3 files changed, 51 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/2449db50/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormatPBINode.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormatPBINode.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormatPBINode.java
index 1c37b62..1ecd947 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormatPBINode.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormatPBINode.java
@@ -77,11 +77,11 @@ public final class FSImageFormatPBINode {
   private final static int USER_STRID_OFFSET = 40;
   private final static int GROUP_STRID_OFFSET = 16;
 
-  private static final int ACL_ENTRY_NAME_MASK = (1 << 24) - 1;
-  private static final int ACL_ENTRY_NAME_OFFSET = 6;
-  private static final int ACL_ENTRY_TYPE_OFFSET = 3;
-  private static final int ACL_ENTRY_SCOPE_OFFSET = 5;
-  private static final int ACL_ENTRY_PERM_MASK = 7;
+  public static final int ACL_ENTRY_NAME_MASK = (1 << 24) - 1;
+  public static final int ACL_ENTRY_NAME_OFFSET = 6;
+  public static final int ACL_ENTRY_TYPE_OFFSET = 3;
+  public static final int ACL_ENTRY_SCOPE_OFFSET = 5;
+  public static final int ACL_ENTRY_PERM_MASK = 7;
   private static final int ACL_ENTRY_TYPE_MASK = 3;
   private static final int ACL_ENTRY_SCOPE_MASK = 1;
   private static final FsAction[] FSACTION_VALUES = FsAction.values();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2449db50/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/OfflineImageReconstructor.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/OfflineImageReconstructor.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/OfflineImageReconstructor.java
index 5c71399..b24ac8e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/OfflineImageReconstructor.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/OfflineImageReconstructor.java
@@ -17,6 +17,10 @@
  */
 package org.apache.hadoop.hdfs.tools.offlineImageViewer;
 
+import static org.apache.hadoop.hdfs.server.namenode.FSImageFormatPBINode.ACL_ENTRY_NAME_MASK;
+import static org.apache.hadoop.hdfs.server.namenode.FSImageFormatPBINode.ACL_ENTRY_NAME_OFFSET;
+import static org.apache.hadoop.hdfs.server.namenode.FSImageFormatPBINode.ACL_ENTRY_SCOPE_OFFSET;
+import static org.apache.hadoop.hdfs.server.namenode.FSImageFormatPBINode.ACL_ENTRY_TYPE_OFFSET;
 import static org.apache.hadoop.hdfs.server.namenode.FSImageFormatPBINode.XATTR_NAMESPACE_MASK;
 import static org.apache.hadoop.hdfs.server.namenode.FSImageFormatPBINode.XATTR_NAME_OFFSET;
 import static org.apache.hadoop.hdfs.server.namenode.FSImageFormatPBINode.XATTR_NAMESPACE_OFFSET;
@@ -49,10 +53,12 @@ import com.google.common.io.CountingOutputStream;
 import com.google.common.primitives.Ints;
 import com.google.protobuf.ByteString;
 import com.google.protobuf.TextFormat;
+
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.fs.permission.AclEntry;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CacheDirectiveInfoExpirationProto;
@@ -66,6 +72,7 @@ import org.apache.hadoop.hdfs.server.namenode.FsImageProto.CacheManagerSection;
 import org.apache.hadoop.hdfs.server.namenode.FsImageProto.FileSummary;
 import org.apache.hadoop.hdfs.server.namenode.FsImageProto.FilesUnderConstructionSection.FileUnderConstructionEntry;
 import org.apache.hadoop.hdfs.server.namenode.FsImageProto.INodeSection;
+import org.apache.hadoop.hdfs.server.namenode.FsImageProto.INodeSection.AclFeatureProto;
 import org.apache.hadoop.hdfs.server.namenode.FsImageProto.NameSystemSection;
 import org.apache.hadoop.hdfs.server.namenode.FsImageProto.SecretManagerSection;
 import org.apache.hadoop.hdfs.server.namenode.FsImageProto.SnapshotDiffSection.DiffEntry;
@@ -131,6 +138,8 @@ class OfflineImageReconstructor {
    */
   private int latestStringId = 0;
 
+  private static final String EMPTY_STRING = "";
+
   private OfflineImageReconstructor(CountingOutputStream out,
       InputStreamReader reader) throws XMLStreamException {
     this.out = out;
@@ -731,10 +740,25 @@ class OfflineImageReconstructor {
     // Will check remaining keys and serialize in processINodeXml
   }
 
-  private INodeSection.AclFeatureProto.Builder aclXmlToProto(Node acl)
+  private INodeSection.AclFeatureProto.Builder aclXmlToProto(Node acls)
       throws IOException {
-    // TODO: support ACLs
-    throw new IOException("ACLs are not supported yet.");
+    AclFeatureProto.Builder b = AclFeatureProto.newBuilder();
+    while (true) {
+      Node acl = acls.removeChild(INODE_SECTION_ACL);
+      if (acl == null) {
+        break;
+      }
+      String val = acl.getVal();
+      AclEntry entry = AclEntry.parseAclEntry(val, true);
+      int nameId = registerStringId(entry.getName() == null ? EMPTY_STRING
+          : entry.getName());
+      int v = ((nameId & ACL_ENTRY_NAME_MASK) << ACL_ENTRY_NAME_OFFSET)
+          | (entry.getType().ordinal() << ACL_ENTRY_TYPE_OFFSET)
+          | (entry.getScope().ordinal() << ACL_ENTRY_SCOPE_OFFSET)
+          | (entry.getPermission().ordinal());
+      b.addEntries(v);
+    }
+    return b;
   }
 
   private INodeSection.XAttrFeatureProto.Builder xattrsXmlToProto(Node xattrs)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2449db50/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/TestOfflineImageViewer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/TestOfflineImageViewer.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/TestOfflineImageViewer.java
index 5d5fea6..1cc82a1 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/TestOfflineImageViewer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/TestOfflineImageViewer.java
@@ -17,6 +17,14 @@
  */
 package org.apache.hadoop.hdfs.tools.offlineImageViewer;
 
+import static org.apache.hadoop.fs.permission.AclEntryScope.ACCESS;
+import static org.apache.hadoop.fs.permission.AclEntryType.GROUP;
+import static org.apache.hadoop.fs.permission.AclEntryType.OTHER;
+import static org.apache.hadoop.fs.permission.AclEntryType.USER;
+import static org.apache.hadoop.fs.permission.FsAction.ALL;
+import static org.apache.hadoop.fs.permission.FsAction.EXECUTE;
+import static org.apache.hadoop.fs.permission.FsAction.READ_EXECUTE;
+import static org.apache.hadoop.hdfs.server.namenode.AclTestHelpers.aclEntry;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 
@@ -49,6 +57,7 @@ import javax.xml.parsers.SAXParser;
 import javax.xml.parsers.SAXParserFactory;
 
 import com.google.common.io.Files;
+
 import org.apache.commons.io.FileUtils;
 import org.apache.commons.io.output.NullOutputStream;
 import org.apache.commons.logging.Log;
@@ -82,6 +91,7 @@ import org.xml.sax.InputSource;
 import org.xml.sax.SAXException;
 import org.xml.sax.helpers.DefaultHandler;
 
+import com.google.common.collect.Lists;
 import com.google.common.collect.Maps;
 
 public class TestOfflineImageViewer {
@@ -113,6 +123,7 @@ public class TestOfflineImageViewer {
           DFSConfigKeys.DFS_NAMENODE_DELEGATION_TOKEN_RENEW_INTERVAL_KEY, 5000);
       conf.setBoolean(
           DFSConfigKeys.DFS_NAMENODE_DELEGATION_TOKEN_ALWAYS_USE_KEY, true);
+      conf.setBoolean(DFSConfigKeys.DFS_NAMENODE_ACLS_ENABLED_KEY, true);
       conf.set(CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTH_TO_LOCAL,
           "RULE:[2:$1@$0](JobTracker@.*FOO.COM)s/@.*//" + "DEFAULT");
       cluster = new MiniDFSCluster.Builder(conf).numDataNodes(1).build();
@@ -179,6 +190,14 @@ public class TestOfflineImageViewer {
       // as UTF8
       hdfs.setXAttr(xattr, "user.a4", new byte[]{ -0x3d, 0x28 });
       writtenFiles.put(xattr.toString(), hdfs.getFileStatus(xattr));
+      // Set ACLs
+      hdfs.setAcl(
+          xattr,
+          Lists.newArrayList(aclEntry(ACCESS, USER, ALL),
+              aclEntry(ACCESS, USER, "foo", ALL),
+              aclEntry(ACCESS, GROUP, READ_EXECUTE),
+              aclEntry(ACCESS, GROUP, "bar", READ_EXECUTE),
+              aclEntry(ACCESS, OTHER, EXECUTE)));
 
       // Write results to the fsimage file
       hdfs.setSafeMode(SafeModeAction.SAFEMODE_ENTER, false);


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


[26/39] hadoop git commit: HADOOP-13255. KMSClientProvider should check and renew tgt when doing delegation token operations. Contributed by Xiao Chen.

Posted by ae...@apache.org.
HADOOP-13255. KMSClientProvider should check and renew tgt when doing delegation token operations. Contributed by Xiao Chen.


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

Branch: refs/heads/HDFS-1312
Commit: b1674caa409ca2c616207acb72aeb2767d28b10c
Parents: 127d2c7
Author: Xiaoyu Yao <xy...@apache.org>
Authored: Thu Jun 16 15:22:00 2016 -0700
Committer: Xiaoyu Yao <xy...@apache.org>
Committed: Thu Jun 16 15:22:00 2016 -0700

----------------------------------------------------------------------
 .../crypto/key/kms/KMSClientProvider.java       |  2 -
 .../hadoop/security/UserGroupInformation.java   |  2 +-
 .../web/DelegationTokenAuthenticator.java       |  3 +
 .../hadoop/crypto/key/kms/server/TestKMS.java   | 91 +++++++++++++++++---
 .../src/test/resources/log4j.properties         |  2 +-
 .../java/org/apache/hadoop/minikdc/MiniKdc.java | 11 ++-
 6 files changed, 95 insertions(+), 16 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/b1674caa/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 f4103b4..7e06ddd 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
@@ -536,8 +536,6 @@ public class KMSClientProvider extends KeyProvider implements CryptoExtension,
           UserGroupInformation.AuthenticationMethod.PROXY)
                               ? currentUgi.getShortUserName() : null;
 
-      // check and renew TGT to handle potential expiration
-      actualUgi.checkTGTAndReloginFromKeytab();
       // creating the HTTP connection using the current UGI at constructor time
       conn = actualUgi.doAs(new PrivilegedExceptionAction<HttpURLConnection>() {
         @Override

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b1674caa/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/UserGroupInformation.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/UserGroupInformation.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/UserGroupInformation.java
index 798aa01..93822a1 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/UserGroupInformation.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/UserGroupInformation.java
@@ -103,7 +103,7 @@ public class UserGroupInformation {
    * @param immediate true if we should login without waiting for ticket window
    */
   @VisibleForTesting
-  static void setShouldRenewImmediatelyForTests(boolean immediate) {
+  public static void setShouldRenewImmediatelyForTests(boolean immediate) {
     shouldRenewImmediatelyForTests = immediate;
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b1674caa/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/web/DelegationTokenAuthenticator.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/web/DelegationTokenAuthenticator.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/web/DelegationTokenAuthenticator.java
index 46a0b1f..53978a6 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/web/DelegationTokenAuthenticator.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/web/DelegationTokenAuthenticator.java
@@ -20,6 +20,7 @@ package org.apache.hadoop.security.token.delegation.web;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.security.SecurityUtil;
+import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.authentication.client.AuthenticatedURL;
 import org.apache.hadoop.security.authentication.client.AuthenticationException;
 import org.apache.hadoop.security.authentication.client.Authenticator;
@@ -143,6 +144,8 @@ public abstract class DelegationTokenAuthenticator implements Authenticator {
   public void authenticate(URL url, AuthenticatedURL.Token token)
       throws IOException, AuthenticationException {
     if (!hasDelegationToken(url, token)) {
+      // check and renew TGT to handle potential expiration
+      UserGroupInformation.getCurrentUser().checkTGTAndReloginFromKeytab();
       authenticator.authenticate(url, token);
     }
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b1674caa/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 db34aa9..94b9d06 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
@@ -42,12 +42,9 @@ import org.apache.hadoop.security.authentication.client.PseudoAuthenticator;
 import org.apache.hadoop.security.authorize.AuthorizationException;
 import org.apache.hadoop.security.ssl.KeyStoreTestUtil;
 import org.apache.hadoop.security.token.Token;
-import org.apache.hadoop.test.GenericTestUtils;
-import org.apache.log4j.Level;
-import org.junit.AfterClass;
+import org.junit.After;
 import org.junit.Assert;
 import org.junit.Before;
-import org.junit.BeforeClass;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.rules.Timeout;
@@ -88,11 +85,11 @@ public class TestKMS {
   public final Timeout testTimeout = new Timeout(180000);
 
   @Before
-  public void cleanUp() {
+  public void setUp() throws Exception {
+    setUpMiniKdc();
     // resetting kerberos security
     Configuration conf = new Configuration();
     UserGroupInformation.setConfiguration(conf);
-    GenericTestUtils.setLogLevel(LOG, Level.INFO);
   }
 
   public static File getTestDir() throws Exception {
@@ -232,10 +229,8 @@ public class TestKMS {
   private static MiniKdc kdc;
   private static File keytab;
 
-  @BeforeClass
-  public static void setUpMiniKdc() throws Exception {
+  private static void setUpMiniKdc(Properties kdcConf) throws Exception {
     File kdcDir = getTestDir();
-    Properties kdcConf = MiniKdc.createConf();
     kdc = new MiniKdc(kdcConf, kdcDir);
     kdc.start();
     keytab = new File(kdcDir, "keytab");
@@ -255,11 +250,18 @@ public class TestKMS {
         principals.toArray(new String[principals.size()]));
   }
 
-  @AfterClass
-  public static void tearDownMiniKdc() throws Exception {
+  private void setUpMiniKdc() throws Exception {
+    Properties kdcConf = MiniKdc.createConf();
+    setUpMiniKdc(kdcConf);
+  }
+
+  @After
+  public void tearDownMiniKdc() throws Exception {
     if (kdc != null) {
       kdc.stop();
+      kdc = null;
     }
+    UserGroupInformation.setShouldRenewImmediatelyForTests(false);
   }
 
   private <T> T doAs(String user, final PrivilegedExceptionAction<T> action)
@@ -2053,6 +2055,73 @@ public class TestKMS {
     doWebHDFSProxyUserTest(false);
   }
 
+  @Test
+  public void testTGTRenewal() throws Exception {
+    tearDownMiniKdc();
+    Properties kdcConf = MiniKdc.createConf();
+    kdcConf.setProperty(MiniKdc.MAX_TICKET_LIFETIME, "3");
+    kdcConf.setProperty(MiniKdc.MIN_TICKET_LIFETIME, "3");
+    setUpMiniKdc(kdcConf);
+
+    Configuration conf = new Configuration();
+    conf.set("hadoop.security.authentication", "kerberos");
+    UserGroupInformation.setConfiguration(conf);
+    final File testDir = getTestDir();
+    conf = createBaseKMSConf(testDir);
+    conf.set("hadoop.kms.authentication.type", "kerberos");
+    conf.set("hadoop.kms.authentication.kerberos.keytab",
+        keytab.getAbsolutePath());
+    conf.set("hadoop.kms.authentication.kerberos.principal", "HTTP/localhost");
+    conf.set("hadoop.kms.authentication.kerberos.name.rules", "DEFAULT");
+    conf.set("hadoop.kms.proxyuser.client.users", "*");
+    conf.set("hadoop.kms.proxyuser.client.hosts", "*");
+    writeConf(testDir, conf);
+
+    runServer(null, null, testDir, new KMSCallable<Void>() {
+      @Override
+      public Void call() throws Exception {
+        final Configuration conf = new Configuration();
+        final URI uri = createKMSUri(getKMSUrl());
+        UserGroupInformation.setShouldRenewImmediatelyForTests(true);
+        UserGroupInformation
+            .loginUserFromKeytab("client", keytab.getAbsolutePath());
+        final UserGroupInformation clientUgi =
+            UserGroupInformation.getCurrentUser();
+        clientUgi.doAs(new PrivilegedExceptionAction<Void>() {
+          @Override
+          public Void run() throws Exception {
+            // Verify getKeys can relogin
+            Thread.sleep(3100);
+            KeyProvider kp = createProvider(uri, conf);
+            kp.getKeys();
+
+            // Verify addDelegationTokens can relogin
+            // (different code path inside KMSClientProvider than getKeys)
+            Thread.sleep(3100);
+            kp = createProvider(uri, conf);
+            ((KeyProviderDelegationTokenExtension.DelegationTokenExtension) kp)
+                .addDelegationTokens("myuser", new Credentials());
+
+            // Verify getKeys can relogin with proxy user
+            UserGroupInformation anotherUgi =
+                UserGroupInformation.createProxyUser("client1", clientUgi);
+            anotherUgi.doAs(new PrivilegedExceptionAction<Void>() {
+              @Override
+              public Void run() throws Exception {
+                Thread.sleep(3100);
+                KeyProvider kp = createProvider(uri, conf);
+                kp.getKeys();
+                return null;
+              }
+            });
+            return null;
+          }
+        });
+        return null;
+      }
+    });
+  }
+
   public void doWebHDFSProxyUserTest(final boolean kerberos) throws Exception {
     Configuration conf = new Configuration();
     conf.set("hadoop.security.authentication", "kerberos");

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b1674caa/hadoop-common-project/hadoop-kms/src/test/resources/log4j.properties
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-kms/src/test/resources/log4j.properties b/hadoop-common-project/hadoop-kms/src/test/resources/log4j.properties
index 5cd037a..b347d27 100644
--- a/hadoop-common-project/hadoop-kms/src/test/resources/log4j.properties
+++ b/hadoop-common-project/hadoop-kms/src/test/resources/log4j.properties
@@ -22,7 +22,7 @@ log4j.appender.stdout.Target=System.out
 log4j.appender.stdout.layout=org.apache.log4j.PatternLayout
 log4j.appender.stdout.layout.ConversionPattern=%d{ISO8601} %-5p %c{1} - %m%n
 
-log4j.rootLogger=WARN, stdout
+log4j.rootLogger=INFO, stdout
 log4j.logger.org.apache.hadoop.conf=ERROR
 log4j.logger.org.apache.hadoop.crytpo.key.kms.server=ALL
 log4j.logger.com.sun.jersey.server.wadl.generators.WadlGeneratorJAXBGrammarGenerator=OFF

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b1674caa/hadoop-common-project/hadoop-minikdc/src/main/java/org/apache/hadoop/minikdc/MiniKdc.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-minikdc/src/main/java/org/apache/hadoop/minikdc/MiniKdc.java b/hadoop-common-project/hadoop-minikdc/src/main/java/org/apache/hadoop/minikdc/MiniKdc.java
index 9278642..281b3cc 100644
--- a/hadoop-common-project/hadoop-minikdc/src/main/java/org/apache/hadoop/minikdc/MiniKdc.java
+++ b/hadoop-common-project/hadoop-minikdc/src/main/java/org/apache/hadoop/minikdc/MiniKdc.java
@@ -147,6 +147,7 @@ public class MiniKdc {
   public static final String KDC_PORT = "kdc.port";
   public static final String INSTANCE = "instance";
   public static final String MAX_TICKET_LIFETIME = "max.ticket.lifetime";
+  public static final String MIN_TICKET_LIFETIME = "min.ticket.lifetime";
   public static final String MAX_RENEWABLE_LIFETIME = "max.renewable.lifetime";
   public static final String TRANSPORT = "transport";
   public static final String DEBUG = "debug";
@@ -280,7 +281,7 @@ public class MiniKdc {
     simpleKdc.init();
     resetDefaultRealm();
     simpleKdc.start();
-    LOG.info("MiniKdc stated.");
+    LOG.info("MiniKdc started.");
   }
 
   private void resetDefaultRealm() throws IOException {
@@ -321,6 +322,14 @@ public class MiniKdc {
     if (conf.getProperty(DEBUG) != null) {
       krb5Debug = getAndSet(SUN_SECURITY_KRB5_DEBUG, conf.getProperty(DEBUG));
     }
+    if (conf.getProperty(MIN_TICKET_LIFETIME) != null) {
+      simpleKdc.getKdcConfig().setLong(KdcConfigKey.MINIMUM_TICKET_LIFETIME,
+          Long.parseLong(conf.getProperty(MIN_TICKET_LIFETIME)));
+    }
+    if (conf.getProperty(MAX_TICKET_LIFETIME) != null) {
+      simpleKdc.getKdcConfig().setLong(KdcConfigKey.MAXIMUM_TICKET_LIFETIME,
+          Long.parseLong(conf.getProperty(MiniKdc.MAX_TICKET_LIFETIME)));
+    }
   }
 
   /**


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


[38/39] hadoop git commit: HDFS-10437. ReconfigurationProtocol not covered by HDFSPolicyProvider. (Arpit Agarwal)

Posted by ae...@apache.org.
HDFS-10437. ReconfigurationProtocol not covered by HDFSPolicyProvider. (Arpit Agarwal)


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

Branch: refs/heads/HDFS-1312
Commit: 0319d73c3b14c09da0f28371dc857b2cc5f6ec7a
Parents: 5f6bc65
Author: Arpit Agarwal <ar...@apache.org>
Authored: Sat Jun 18 16:16:54 2016 -0700
Committer: Arpit Agarwal <ar...@apache.org>
Committed: Sat Jun 18 16:16:54 2016 -0700

----------------------------------------------------------------------
 .../hadoop/fs/CommonConfigurationKeys.java      |  3 ++
 .../apache/hadoop/hdfs/HDFSPolicyProvider.java  |  6 +++-
 .../hadoop/hdfs/TestHDFSPolicyProvider.java     | 31 +++++++++-----------
 3 files changed, 22 insertions(+), 18 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/0319d73c/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 06614db..e455201 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
@@ -207,6 +207,9 @@ public class CommonConfigurationKeys extends CommonConfigurationKeysPublic {
   public static final String
       HADOOP_SECURITY_SERVICE_AUTHORIZATION_DATANODE_LIFELINE =
           "security.datanode.lifeline.protocol.acl";
+  public static final String
+      HADOOP_SECURITY_SERVICE_AUTHORIZATION_RECONFIGURATION =
+      "security.reconfiguration.protocol.acl";
   public static final String 
   SECURITY_HA_SERVICE_PROTOCOL_ACL = "security.ha.service.protocol.acl";
   public static final String 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0319d73c/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/HDFSPolicyProvider.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/HDFSPolicyProvider.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/HDFSPolicyProvider.java
index 8c20553..edc72c5 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/HDFSPolicyProvider.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/HDFSPolicyProvider.java
@@ -23,6 +23,7 @@ import org.apache.hadoop.ha.HAServiceProtocol;
 import org.apache.hadoop.ha.ZKFCProtocol;
 import org.apache.hadoop.hdfs.protocol.ClientDatanodeProtocol;
 import org.apache.hadoop.hdfs.protocol.ClientProtocol;
+import org.apache.hadoop.hdfs.protocol.ReconfigurationProtocol;
 import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocol;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeLifelineProtocol;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeProtocol;
@@ -80,7 +81,10 @@ public class HDFSPolicyProvider extends PolicyProvider {
         TraceAdminProtocol.class),
     new Service(
         CommonConfigurationKeys.HADOOP_SECURITY_SERVICE_AUTHORIZATION_DATANODE_LIFELINE,
-        DatanodeLifelineProtocol.class)
+        DatanodeLifelineProtocol.class),
+    new Service(
+          CommonConfigurationKeys.HADOOP_SECURITY_SERVICE_AUTHORIZATION_RECONFIGURATION,
+          ReconfigurationProtocol.class)
   };
   
   @Override

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0319d73c/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestHDFSPolicyProvider.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestHDFSPolicyProvider.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestHDFSPolicyProvider.java
index 95aa89f..e242690 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestHDFSPolicyProvider.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestHDFSPolicyProvider.java
@@ -23,10 +23,12 @@ import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collections;
 import java.util.Comparator;
+import java.util.HashSet;
 import java.util.List;
+import java.util.Set;
 
+import com.google.common.collect.Sets;
 import org.apache.commons.lang.ClassUtils;
-import org.apache.hadoop.hdfs.protocol.ReconfigurationProtocol;
 import org.apache.hadoop.hdfs.qjournal.server.JournalNodeRpcServer;
 import org.apache.hadoop.hdfs.server.namenode.NameNodeRpcServer;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
@@ -57,7 +59,7 @@ public class TestHDFSPolicyProvider {
   private static final Logger LOG =
       LoggerFactory.getLogger(TestHDFSPolicyProvider.class);
 
-  private static List<Class<?>> policyProviderProtocols;
+  private static Set<Class<?>> policyProviderProtocols;
 
   private static final Comparator<Class<?>> CLASS_NAME_COMPARATOR =
       new Comparator<Class<?>>() {
@@ -75,11 +77,10 @@ public class TestHDFSPolicyProvider {
   @BeforeClass
   public static void initialize() {
     Service[] services = new HDFSPolicyProvider().getServices();
-    policyProviderProtocols = new ArrayList<>(services.length);
+    policyProviderProtocols = new HashSet<>(services.length);
     for (Service service : services) {
       policyProviderProtocols.add(service.getProtocol());
     }
-    Collections.sort(policyProviderProtocols, CLASS_NAME_COMPARATOR);
   }
 
   public TestHDFSPolicyProvider(Class<?> rpcServerClass) {
@@ -98,29 +99,25 @@ public class TestHDFSPolicyProvider {
   @Test
   public void testPolicyProviderForServer() {
     List<?> ifaces = ClassUtils.getAllInterfaces(rpcServerClass);
-    List<Class<?>> serverProtocols = new ArrayList<>(ifaces.size());
+    Set<Class<?>> serverProtocols = new HashSet<>(ifaces.size());
     for (Object obj : ifaces) {
       Class<?> iface = (Class<?>)obj;
-      // ReconfigurationProtocol is not covered in HDFSPolicyProvider
-      // currently, so we have a special case to skip it.  This needs follow-up
-      // investigation.
-      if (iface.getSimpleName().endsWith("Protocol") &&
-          iface != ReconfigurationProtocol.class) {
+      if (iface.getSimpleName().endsWith("Protocol")) {
         serverProtocols.add(iface);
       }
     }
-    Collections.sort(serverProtocols, CLASS_NAME_COMPARATOR);
     LOG.info("Running test {} for RPC server {}.  Found server protocols {} "
         + "and policy provider protocols {}.", testName.getMethodName(),
         rpcServerClass.getName(), serverProtocols, policyProviderProtocols);
     assertFalse("Expected to find at least one protocol in server.",
         serverProtocols.isEmpty());
+    final Set<Class<?>> differenceSet =
+        Sets.difference(serverProtocols, policyProviderProtocols);
     assertTrue(
-        String.format("Expected all protocols for server %s to be defined in "
-            + "%s.  Server contains protocols %s.  Policy provider contains "
-            + "protocols %s.", rpcServerClass.getName(),
-            HDFSPolicyProvider.class.getName(), serverProtocols,
-            policyProviderProtocols),
-        policyProviderProtocols.containsAll(serverProtocols));
+        String.format("Following protocols for server %s are not defined in "
+            + "%s: %s",
+            rpcServerClass.getName(), HDFSPolicyProvider.class.getName(),
+            Arrays.toString(differenceSet.toArray())),
+        differenceSet.isEmpty());
   }
 }


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


[16/39] hadoop git commit: HDFS-9922. Upgrade Domain placement policy status marks a good block in violation when there are decommissioned nodes. (Chris Trezzo via mingma)

Posted by ae...@apache.org.
HDFS-9922. Upgrade Domain placement policy status marks a good block in violation when there are decommissioned nodes. (Chris Trezzo via mingma)


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

Branch: refs/heads/HDFS-1312
Commit: b48f27e794e42ba90836314834e872616437d7c9
Parents: 5dfc38f
Author: Ming Ma <mi...@apache.org>
Authored: Wed Jun 15 22:00:52 2016 -0700
Committer: Ming Ma <mi...@apache.org>
Committed: Wed Jun 15 22:00:52 2016 -0700

----------------------------------------------------------------------
 .../BlockPlacementStatusWithUpgradeDomain.java  |   2 +-
 ...stBlockPlacementStatusWithUpgradeDomain.java |  83 ++++++++++
 .../TestUpgradeDomainBlockPlacementPolicy.java  | 161 ++++++++++++++-----
 3 files changed, 209 insertions(+), 37 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/b48f27e7/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementStatusWithUpgradeDomain.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementStatusWithUpgradeDomain.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementStatusWithUpgradeDomain.java
index e2e1486..4b3c3cc 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementStatusWithUpgradeDomain.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementStatusWithUpgradeDomain.java
@@ -60,7 +60,7 @@ public class BlockPlacementStatusWithUpgradeDomain implements
 
   private boolean isUpgradeDomainPolicySatisfied() {
     if (numberOfReplicas <= upgradeDomainFactor) {
-      return (numberOfReplicas == upgradeDomains.size());
+      return (numberOfReplicas <= upgradeDomains.size());
     } else {
       return upgradeDomains.size() >= upgradeDomainFactor;
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b48f27e7/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockPlacementStatusWithUpgradeDomain.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockPlacementStatusWithUpgradeDomain.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockPlacementStatusWithUpgradeDomain.java
new file mode 100644
index 0000000..bfff932
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockPlacementStatusWithUpgradeDomain.java
@@ -0,0 +1,83 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.server.blockmanagement;
+
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+import java.util.HashSet;
+import java.util.Set;
+
+import org.junit.Before;
+import org.junit.Test;
+
+/**
+ * Unit tests for BlockPlacementStatusWithUpgradeDomain class.
+ */
+public class TestBlockPlacementStatusWithUpgradeDomain {
+
+  private Set<String> upgradeDomains;
+  private BlockPlacementStatusDefault bpsd =
+      mock(BlockPlacementStatusDefault.class);
+
+  @Before
+  public void setup() {
+    upgradeDomains = new HashSet<String>();
+    upgradeDomains.add("1");
+    upgradeDomains.add("2");
+    upgradeDomains.add("3");
+    when(bpsd.isPlacementPolicySatisfied()).thenReturn(true);
+  }
+
+  @Test
+  public void testIsPolicySatisfiedParentFalse() {
+    when(bpsd.isPlacementPolicySatisfied()).thenReturn(false);
+    BlockPlacementStatusWithUpgradeDomain bps =
+        new BlockPlacementStatusWithUpgradeDomain(bpsd, upgradeDomains, 3, 3);
+
+    // Parent policy is not satisfied but upgrade domain policy is
+    assertFalse(bps.isPlacementPolicySatisfied());
+  }
+
+  @Test
+  public void testIsPolicySatisfiedAllEqual() {
+    BlockPlacementStatusWithUpgradeDomain bps =
+        new BlockPlacementStatusWithUpgradeDomain(bpsd, upgradeDomains, 3, 3);
+    // Number of domains, replicas and upgradeDomainFactor is equal and parent
+    // policy is satisfied
+    assertTrue(bps.isPlacementPolicySatisfied());
+  }
+
+  @Test
+  public void testIsPolicySatisifedSmallDomains() {
+    // Number of domains is less than replicas but equal to factor
+    BlockPlacementStatusWithUpgradeDomain bps =
+        new BlockPlacementStatusWithUpgradeDomain(bpsd, upgradeDomains, 4, 3);
+    assertTrue(bps.isPlacementPolicySatisfied());
+
+    // Same as above but replicas is greater than factor
+    bps = new BlockPlacementStatusWithUpgradeDomain(bpsd, upgradeDomains, 4, 2);
+    assertTrue(bps.isPlacementPolicySatisfied());
+
+    // Number of domains is less than replicas and factor
+    bps = new BlockPlacementStatusWithUpgradeDomain(bpsd, upgradeDomains, 4, 4);
+    assertFalse(bps.isPlacementPolicySatisfied());
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b48f27e7/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestUpgradeDomainBlockPlacementPolicy.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestUpgradeDomainBlockPlacementPolicy.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestUpgradeDomainBlockPlacementPolicy.java
index f9a2503..8460b6f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestUpgradeDomainBlockPlacementPolicy.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestUpgradeDomainBlockPlacementPolicy.java
@@ -17,38 +17,40 @@
  */
 package org.apache.hadoop.hdfs.server.namenode;
 
-import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 
 import java.io.IOException;
-import java.util.Arrays;
-import java.util.EnumSet;
 import java.util.HashSet;
 import java.util.Set;
 
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.CreateFlag;
+import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.permission.PermissionStatus;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.protocol.DatanodeAdminProperties;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
-import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
+import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockPlacementPolicy;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockPlacementPolicyWithUpgradeDomain;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockPlacementStatus;
 import org.apache.hadoop.hdfs.server.blockmanagement.CombinedHostFileManager;
 import org.apache.hadoop.hdfs.server.blockmanagement.HostConfigManager;
 import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocols;
 import org.apache.hadoop.hdfs.util.HostsFileWriter;
 import org.apache.hadoop.net.StaticMapping;
+import org.apache.hadoop.test.GenericTestUtils;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
 
+import com.google.common.base.Supplier;
+
 /**
  * End-to-end test case for upgrade domain
  * The test configs upgrade domain for nodes via admin json
@@ -69,15 +71,16 @@ public class TestUpgradeDomainBlockPlacementPolicy {
    *  CombinedHostFileManager won't allow those hosts.
    */
   static final String[] hosts =
-      { "127.0.0.1", "127.0.0.1", "127.0.0.1", "127.0.0.1",
-          "127.0.0.1", "127.0.0.1" };
+      {"127.0.0.1", "127.0.0.1", "127.0.0.1", "127.0.0.1",
+          "127.0.0.1", "127.0.0.1"};
   static final String[] upgradeDomains =
-      { "ud1", "ud2", "ud3", "ud1", "ud2", "ud3" };
+      {"ud5", "ud2", "ud3", "ud1", "ud2", "ud4"};
   static final Set<DatanodeID> expectedDatanodeIDs = new HashSet<>();
   private MiniDFSCluster cluster = null;
   private NamenodeProtocols nameNodeRpc = null;
   private FSNamesystem namesystem = null;
   private PermissionStatus perm = null;
+  private HostsFileWriter hostsFileWriter = new HostsFileWriter();
 
   @Before
   public void setup() throws IOException {
@@ -86,11 +89,10 @@ public class TestUpgradeDomainBlockPlacementPolicy {
     conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, DEFAULT_BLOCK_SIZE);
     conf.setInt(DFSConfigKeys.DFS_BYTES_PER_CHECKSUM_KEY, DEFAULT_BLOCK_SIZE / 2);
     conf.setClass(DFSConfigKeys.DFS_BLOCK_REPLICATOR_CLASSNAME_KEY,
-        BlockPlacementPolicyWithUpgradeDomain.class,
-        BlockPlacementPolicy.class);
+            BlockPlacementPolicyWithUpgradeDomain.class,
+            BlockPlacementPolicy.class);
     conf.setClass(DFSConfigKeys.DFS_NAMENODE_HOSTS_PROVIDER_CLASSNAME_KEY,
-        CombinedHostFileManager.class, HostConfigManager.class);
-    HostsFileWriter hostsFileWriter = new HostsFileWriter();
+            CombinedHostFileManager.class, HostConfigManager.class);
     hostsFileWriter.initialize(conf, "temp/upgradedomainpolicy");
 
     cluster = new MiniDFSCluster.Builder(conf).numDataNodes(6).racks(racks)
@@ -100,12 +102,12 @@ public class TestUpgradeDomainBlockPlacementPolicy {
     namesystem = cluster.getNamesystem();
     perm = new PermissionStatus("TestDefaultBlockPlacementPolicy", null,
         FsPermission.getDefault());
-    refreshDatanodeAdminProperties(hostsFileWriter);
-    hostsFileWriter.cleanup();
+    refreshDatanodeAdminProperties();
   }
 
   @After
-  public void teardown() {
+  public void teardown() throws IOException {
+    hostsFileWriter.cleanup();
     if (cluster != null) {
       cluster.shutdown();
       cluster = null;
@@ -114,15 +116,18 @@ public class TestUpgradeDomainBlockPlacementPolicy {
 
   /**
    * Define admin properties for these datanodes as follows.
-   * dn0 and dn3 have upgrade domain ud1.
-   * dn1 and dn4 have upgrade domain ud2.
-   * dn2 and dn5 have upgrade domain ud3.
+   * dn0's upgrade domain is ud5.
+   * dn1's upgrade domain is ud2.
+   * dn2's upgrade domain is ud3.
+   * dn3's upgrade domain is ud1.
+   * dn4's upgrade domain is ud2.
+   * dn5's upgrade domain is ud4.
    * dn0 and dn5 are decommissioned.
    * Given dn0, dn1 and dn2 are on rack1 and dn3, dn4 and dn5 are on
    * rack2. Then any block's replicas should be on either
    * {dn1, dn2, d3} or {dn2, dn3, dn4}.
    */
-  private void refreshDatanodeAdminProperties(HostsFileWriter hostsFileWriter)
+  private void refreshDatanodeAdminProperties()
       throws IOException {
     DatanodeAdminProperties[] datanodes = new DatanodeAdminProperties[
         hosts.length];
@@ -138,32 +143,116 @@ public class TestUpgradeDomainBlockPlacementPolicy {
     hostsFileWriter.initIncludeHosts(datanodes);
     cluster.getFileSystem().refreshNodes();
 
+    expectedDatanodeIDs.clear();
     expectedDatanodeIDs.add(cluster.getDataNodes().get(2).getDatanodeId());
     expectedDatanodeIDs.add(cluster.getDataNodes().get(3).getDatanodeId());
   }
 
+  /**
+   * Define admin properties for these datanodes as follows.
+   * dn0's upgrade domain is ud5.
+   * dn1's upgrade domain is ud2.
+   * dn2's upgrade domain is ud3.
+   * dn3's upgrade domain is ud1.
+   * dn4's upgrade domain is ud2.
+   * dn5's upgrade domain is ud4.
+   * dn2 and dn3 are decommissioned.
+   * Given dn0, dn1 and dn2 are on rack1 and dn3, dn4 and dn5 are on
+   * rack2. Then any block's replicas should be on either
+   * {dn0, dn1, d5} or {dn0, dn4, dn5}.
+   */
+  private void refreshDatanodeAdminProperties2()
+      throws IOException {
+    DatanodeAdminProperties[] datanodes = new DatanodeAdminProperties[
+        hosts.length];
+    for (int i = 0; i < hosts.length; i++) {
+      datanodes[i] = new DatanodeAdminProperties();
+      DatanodeID datanodeID = cluster.getDataNodes().get(i).getDatanodeId();
+      datanodes[i].setHostName(datanodeID.getHostName());
+      datanodes[i].setPort(datanodeID.getXferPort());
+      datanodes[i].setUpgradeDomain(upgradeDomains[i]);
+    }
+    datanodes[2].setAdminState(DatanodeInfo.AdminStates.DECOMMISSIONED);
+    datanodes[3].setAdminState(DatanodeInfo.AdminStates.DECOMMISSIONED);
+    hostsFileWriter.initIncludeHosts(datanodes);
+    cluster.getFileSystem().refreshNodes();
+
+    expectedDatanodeIDs.clear();
+    expectedDatanodeIDs.add(cluster.getDataNodes().get(0).getDatanodeId());
+    expectedDatanodeIDs.add(cluster.getDataNodes().get(5).getDatanodeId());
+  }
+
   @Test
   public void testPlacement() throws Exception {
-    String clientMachine = "127.0.0.1";
-    for (int i = 0; i < 5; i++) {
-      String src = "/test-" + i;
-      // Create the file with client machine
-      HdfsFileStatus fileStatus = namesystem.startFile(src, perm,
-          clientMachine, clientMachine, EnumSet.of(CreateFlag.CREATE), true,
-          REPLICATION_FACTOR, DEFAULT_BLOCK_SIZE, null, false);
-      LocatedBlock locatedBlock = nameNodeRpc.addBlock(src, clientMachine,
-          null, null, fileStatus.getFileId(), null, null);
-
-      assertEquals("Block should be allocated sufficient locations",
-          REPLICATION_FACTOR, locatedBlock.getLocations().length);
-      Set<DatanodeInfo> locs = new HashSet<>(Arrays.asList(
-          locatedBlock.getLocations()));
+    final long fileSize = DEFAULT_BLOCK_SIZE * 5;
+    final String testFile = new String("/testfile");
+    final Path path = new Path(testFile);
+    DFSTestUtil.createFile(cluster.getFileSystem(), path, fileSize,
+        REPLICATION_FACTOR, 1000L);
+    LocatedBlocks locatedBlocks =
+        cluster.getFileSystem().getClient().getLocatedBlocks(
+            path.toString(), 0, fileSize);
+    for (LocatedBlock block : locatedBlocks.getLocatedBlocks()) {
+      Set<DatanodeInfo> locs = new HashSet<>();
+      for(DatanodeInfo datanodeInfo : block.getLocations()) {
+        if (datanodeInfo.getAdminState() == DatanodeInfo.AdminStates.NORMAL) {
+          locs.add(datanodeInfo);
+        }
+      }
       for (DatanodeID datanodeID : expectedDatanodeIDs) {
-        locs.contains(datanodeID);
+        assertTrue(locs.contains(datanodeID));
+      }
+    }
+  }
+
+  @Test(timeout = 300000)
+  public void testPlacementAfterDecommission() throws Exception {
+    final long fileSize = DEFAULT_BLOCK_SIZE * 5;
+    final String testFile = new String("/testfile");
+    final Path path = new Path(testFile);
+    DFSTestUtil.createFile(cluster.getFileSystem(), path, fileSize,
+        REPLICATION_FACTOR, 1000L);
+
+    // Decommission some nodes and wait until decommissions have finished.
+    refreshDatanodeAdminProperties2();
+    GenericTestUtils.waitFor(new Supplier<Boolean>() {
+      @Override
+      public Boolean get() {
+        boolean successful = true;
+        LocatedBlocks locatedBlocks;
+        try {
+          locatedBlocks =
+              cluster.getFileSystem().getClient().getLocatedBlocks(
+                  path.toString(), 0, fileSize);
+        } catch (IOException ioe) {
+          return false;
+        }
+        for(LocatedBlock block : locatedBlocks.getLocatedBlocks()) {
+          Set<DatanodeInfo> locs = new HashSet<>();
+          for (DatanodeInfo datanodeInfo : block.getLocations()) {
+            if (datanodeInfo.getAdminState() ==
+                DatanodeInfo.AdminStates.NORMAL) {
+              locs.add(datanodeInfo);
+            }
+          }
+          for (DatanodeID datanodeID : expectedDatanodeIDs) {
+            successful = successful && locs.contains(datanodeID);
+          }
+        }
+        return successful;
       }
+    }, 1000, 60000);
 
-      nameNodeRpc.abandonBlock(locatedBlock.getBlock(), fileStatus.getFileId(),
-          src, clientMachine);
+    // Verify block placement policy of each block.
+    LocatedBlocks locatedBlocks;
+    locatedBlocks =
+        cluster.getFileSystem().getClient().getLocatedBlocks(
+            path.toString(), 0, fileSize);
+    for(LocatedBlock block : locatedBlocks.getLocatedBlocks()) {
+      BlockPlacementStatus status = cluster.getNamesystem().getBlockManager().
+          getBlockPlacementPolicy().verifyBlockPlacement(
+              block.getLocations(), REPLICATION_FACTOR);
+      assertTrue(status.isPlacementPolicySatisfied());
     }
   }
 }


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


[11/39] hadoop git commit: HADOOP-12893. Verify LICENSE.txt and NOTICE.txt. Contributed by Xiao Chen, Andrew Wang, and Akira Ajisaka.

Posted by ae...@apache.org.
HADOOP-12893. Verify LICENSE.txt and NOTICE.txt. Contributed by Xiao Chen, Andrew Wang, and Akira Ajisaka.


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

Branch: refs/heads/HDFS-1312
Commit: 098ae11e35e5342fc732087e009a402259f544bb
Parents: 7d521a2
Author: Akira Ajisaka <aa...@apache.org>
Authored: Wed Jun 15 13:30:14 2016 +0900
Committer: Akira Ajisaka <aa...@apache.org>
Committed: Wed Jun 15 13:30:14 2016 +0900

----------------------------------------------------------------------
 LICENSE.txt                 | 1057 +++++++++++++++++++++++++++++++++++++-
 NOTICE.txt                  |  266 ++++++++++
 hadoop-build-tools/pom.xml  |   41 ++
 hadoop-project-dist/pom.xml |    2 +
 hadoop-project/pom.xml      |   19 +-
 pom.xml                     |   26 +
 6 files changed, 1387 insertions(+), 24 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/098ae11e/LICENSE.txt
----------------------------------------------------------------------
diff --git a/LICENSE.txt b/LICENSE.txt
index 929e2a8..92ae3b1 100644
--- a/LICENSE.txt
+++ b/LICENSE.txt
@@ -207,7 +207,7 @@ APACHE HADOOP SUBCOMPONENTS:
 The Apache Hadoop project contains subcomponents with separate copyright
 notices and license terms. Your use of the source code for the these
 subcomponents is subject to the terms and conditions of the following
-licenses. 
+licenses.
 
 For the org.apache.hadoop.util.bloom.* classes:
 
@@ -216,30 +216,30 @@ For the org.apache.hadoop.util.bloom.* classes:
  * Copyright (c) 2005, European Commission project OneLab under contract
  * 034819 (http://www.one-lab.org)
  * All rights reserved.
- * Redistribution and use in source and binary forms, with or 
- * without modification, are permitted provided that the following 
+ * Redistribution and use in source and binary forms, with or
+ * without modification, are permitted provided that the following
  * conditions are met:
- *  - Redistributions of source code must retain the above copyright 
+ *  - Redistributions of source code must retain the above copyright
  *    notice, this list of conditions and the following disclaimer.
- *  - Redistributions in binary form must reproduce the above copyright 
- *    notice, this list of conditions and the following disclaimer in 
+ *  - Redistributions in binary form must reproduce the above copyright
+ *    notice, this list of conditions and the following disclaimer in
  *    the documentation and/or other materials provided with the distribution.
  *  - Neither the name of the University Catholique de Louvain - UCL
- *    nor the names of its contributors may be used to endorse or 
- *    promote products derived from this software without specific prior 
+ *    nor the names of its contributors may be used to endorse or
+ *    promote products derived from this software without specific prior
  *    written permission.
- *    
- * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS 
- * "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT 
- * LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS 
- * FOR A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE 
- * COPYRIGHT OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, 
- * INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, 
- * BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; 
- * LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER 
- * CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT 
- * LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN 
- * ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE 
+ *
+ * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+ * "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+ * LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS
+ * FOR A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE
+ * COPYRIGHT OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT,
+ * INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING,
+ * BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES;
+ * LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER
+ * CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT
+ * LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN
+ * ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE
  * POSSIBILITY OF SUCH DAMAGE.
  */
 
@@ -320,7 +320,9 @@ THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
 (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
 OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
 
-For com.google.re2j.* classes:
+The binary distribution of this product bundles these dependencies under the
+following license:
+re2j 1.0
 ---------------------------------------------------------------------
 This is a work derived from Russ Cox's RE2 in Go, whose license
 http://golang.org/LICENSE is as follows:
@@ -548,12 +550,14 @@ hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/static/bootstrap-3.0.2
 hadoop-tools/hadoop-sls/src/main/html/js/thirdparty/bootstrap.min.js
 hadoop-tools/hadoop-sls/src/main/html/css/bootstrap.min.css
 hadoop-tools/hadoop-sls/src/main/html/css/bootstrap-responsive.min.css
+And the binary distribution of this product bundles these dependencies under the
+following license:
+Mockito 1.8.5
+SLF4J 1.7.10
 --------------------------------------------------------------------------------
 
 The MIT License (MIT)
 
-Copyright (c) 2011-2016 Twitter, Inc.
-
 Permission is hereby granted, free of charge, to any person obtaining a copy
 of this software and associated documentation files (the "Software"), to deal
 in the Software without restriction, including without limitation the rights
@@ -648,3 +652,1010 @@ hadoop-tools/hadoop-sls/src/main/html/js/thirdparty/d3.v3.js
 
 D3 is available under a 3-clause BSD license. For details, see:
 hadoop-tools/hadoop-sls/src/main/html/js/thirdparty/d3-LICENSE
+
+The binary distribution of this product bundles these dependencies under the
+following license:
+HSQLDB Database 2.0.0
+--------------------------------------------------------------------------------
+"COPYRIGHTS AND LICENSES (based on BSD License)
+
+For work developed by the HSQL Development Group:
+
+Copyright (c) 2001-2016, The HSQL Development Group
+All rights reserved.
+
+Redistribution and use in source and binary forms, with or without
+modification, are permitted provided that the following conditions are met:
+
+Redistributions of source code must retain the above copyright notice, this
+list of conditions and the following disclaimer.
+
+Redistributions in binary form must reproduce the above copyright notice,
+this list of conditions and the following disclaimer in the documentation
+and/or other materials provided with the distribution.
+
+Neither the name of the HSQL Development Group nor the names of its
+contributors may be used to endorse or promote products derived from this
+software without specific prior written permission.
+
+THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS ""AS IS""
+AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE
+IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE
+ARE DISCLAIMED. IN NO EVENT SHALL HSQL DEVELOPMENT GROUP, HSQLDB.ORG,
+OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL,
+EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO,
+PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES;
+LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND
+ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS
+SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+
+
+For work originally developed by the Hypersonic SQL Group:
+
+Copyright (c) 1995-2000 by the Hypersonic SQL Group.
+All rights reserved.
+Redistribution and use in source and binary forms, with or without
+modification, are permitted provided that the following conditions are met:
+
+Redistributions of source code must retain the above copyright notice, this
+list of conditions and the following disclaimer.
+
+Redistributions in binary form must reproduce the above copyright notice,
+this list of conditions and the following disclaimer in the documentation
+and/or other materials provided with the distribution.
+
+Neither the name of the Hypersonic SQL Group nor the names of its
+contributors may be used to endorse or promote products derived from this
+software without specific prior written permission.
+
+THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS ""AS IS""
+AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE
+IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE
+ARE DISCLAIMED. IN NO EVENT SHALL THE HYPERSONIC SQL GROUP,
+OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL,
+EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO,
+PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES;
+LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND
+ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS
+SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+
+This software consists of voluntary contributions made by many individuals on behalf of the
+Hypersonic SQL Group."
+
+The binary distribution of this product bundles these dependencies under the
+following license:
+servlet-api 2.5
+jsp-api 2.1
+Streaming API for XML 1.0
+--------------------------------------------------------------------------------
+COMMON DEVELOPMENT AND DISTRIBUTION LICENSE (CDDL) Version 1.0
+1.�Definitions.�
+
+1.1.�Contributor�means each individual or entity
+that creates or contributes to the creation of
+Modifications.�
+
+1.2.�Contributor Version�means the combination of the
+Original Software, prior Modifications used by a Contributor (if any), and the
+Modifications made by that particular Contributor.�
+
+1.3.�Covered
+Software�means (a) the Original Software, or (b) Modifications, or (c) the
+combination of files containing Original Software with files containing
+Modifications, in each case including portions
+thereof.�
+
+1.4.�Executable�means the Covered Software in any form other
+than Source Code.�
+
+1.5.�Initial Developer�means the individual or entity
+that first makes Original Software available under this
+License.�
+
+1.6.�Larger Work�means a work which combines Covered Software or
+portions thereof with code not governed by the terms of this
+License.�
+
+1.7.�License�means this document.�
+
+1.8.�Licensable�means
+having the right to grant, to the maximum extent possible, whether at the time
+of the initial grant or subsequently acquired, any and all of the rights
+conveyed herein.�
+
+1.9.�Modifications�means the Source Code and Executable
+form of any of the following:
+A. Any file that results from an addition to,
+deletion from or modification of the contents of a file containing Original
+Software or previous Modifications;
+B. Any new file that contains any part of the Original Software
+or previous Modification; or
+C. Any new file that is contributed or otherwise made available
+under the terms of this License.�
+
+1.10.�Original Software�means the Source Code and Executable form of
+computer software code that is originally released under this License.�
+
+1.11.�Patent Claims�means any patent claim(s), now owned or
+hereafter acquired, including without limitation, method, process, and apparatus
+claims, in any patent Licensable by grantor.�
+
+1.12.�Source Code�means (a) the common form of computer software code in which
+modifications are made and (b) associated documentation included in or
+with such code.�
+
+1.13.�You (or Your)�means an individual or a legal entity exercising rights
+under, and complying with all of the terms of, this License. For legal entities,
+You includes any entity which controls, is controlled by, or is under common control
+with You. For purposes of this definition, control means (a)�the power, direct
+or indirect, to cause the direction or management of such entity, whether by
+contract or otherwise, or (b)�ownership of more than fifty percent (50%) of the
+outstanding shares or beneficial ownership of such entity.�
+
+2. License Grants.
+
+2.1. The Initial Developer Grant. Conditioned upon Your compliance
+with Section 3.1 below and subject to third party intellectual property claims,
+the Initial Developer hereby grants You a world-wide, royalty-free,
+non-exclusive license:�
+
+(a) under intellectual property rights (other than
+patent or trademark) Licensable by Initial Developer, to use, reproduce, modify,
+display, perform, sublicense and distribute the Original Software (or portions
+thereof), with or without Modifications, and/or as part of a Larger Work;
+and�
+
+(b) under Patent Claims infringed by the making, using or selling of
+Original Software, to make, have made, use, practice, sell, and offer for sale,
+and/or otherwise dispose of the Original Software (or portions
+thereof);
+
+(c) The licenses granted in Sections�2.1(a) and (b) are
+effective on the date Initial Developer first distributes or otherwise makes the
+Original Software available to a third party under the terms of this
+License;
+
+(d) Notwithstanding Section�2.1(b) above, no patent license is
+granted: (1)�for code that You delete from the Original Software, or (2)�for
+infringements caused by: (i)�the modification of the Original Software, or
+(ii)�the combination of the Original Software with other software or
+devices.�
+
+2.2. Contributor Grant. Conditioned upon Your compliance with
+Section 3.1 below and subject to third party intellectual property claims, each
+Contributor hereby grants You a world-wide, royalty-free, non-exclusive
+license:�
+
+(a) under intellectual property rights (other than patent or
+trademark) Licensable by Contributor to use, reproduce, modify, display,
+perform, sublicense and distribute the Modifications created by such Contributor
+(or portions thereof), either on an unmodified basis, with other Modifications,
+as Covered Software and/or as part of a Larger Work; and�
+
+(b) under Patent
+Claims infringed by the making, using, or selling of Modifications made by that
+Contributor either alone and/or in combination with its Contributor Version (or
+portions of such combination), to make, use, sell, offer for sale, have made,
+and/or otherwise dispose of: (1)�Modifications made by that Contributor (or
+portions thereof); and (2)�the combination of Modifications made by that
+Contributor with its Contributor Version (or portions of such
+combination).�
+
+(c) The licenses granted in Sections�2.2(a) and 2.2(b) are
+effective on the date Contributor first distributes or otherwise makes the
+Modifications available to a third party.
+
+(d) Notwithstanding Section�2.2(b)
+above, no patent license is granted: (1)�for any code that Contributor has
+deleted from the Contributor Version; (2)�for infringements caused by:
+(i)�third party modifications of Contributor Version, or (ii)�the combination
+of Modifications made by that Contributor with other software (except as part of
+the Contributor Version) or other devices; or (3)�under Patent Claims infringed
+by Covered Software in the absence of Modifications made by that
+Contributor.�
+
+3. Distribution Obligations.�
+
+3.1. Availability of Source
+Code. Any Covered Software that You distribute or otherwise make available in
+Executable form must also be made available in Source Code form and that Source
+Code form must be distributed only under the terms of this License. You must
+include a copy of this License with every copy of the Source Code form of the
+Covered Software You distribute or otherwise make available. You must inform
+recipients of any such Covered Software in Executable form as to how they can
+obtain such Covered Software in Source Code form in a reasonable manner on or
+through a medium customarily used for software exchange.�
+
+3.2.
+Modifications. The Modifications that You create or to which You contribute are
+governed by the terms of this License. You represent that You believe Your
+Modifications are Your original creation(s) and/or You have sufficient rights to
+grant the rights conveyed by this License.�
+
+3.3. Required Notices. You must
+include a notice in each of Your Modifications that identifies You as the
+Contributor of the Modification. You may not remove or alter any copyright,
+patent or trademark notices contained within the Covered Software, or any
+notices of licensing or any descriptive text giving attribution to any
+Contributor or the Initial Developer.�
+
+3.4. Application of Additional Terms.
+You may not offer or impose any terms on any Covered Software in Source Code
+form that alters or restricts the applicable version of this License or the
+recipients rights hereunder. You may choose to offer, and to charge a fee for,
+warranty, support, indemnity or liability obligations to one or more recipients
+of Covered Software. However, you may do so only on Your own behalf, and not on
+behalf of the Initial Developer or any Contributor. You must make it absolutely
+clear that any such warranty, support, indemnity or liability obligation is
+offered by You alone, and You hereby agree to indemnify the Initial Developer
+and every Contributor for any liability incurred by the Initial Developer or
+such Contributor as a result of warranty, support, indemnity or liability terms
+You offer.
+
+3.5. Distribution of Executable Versions. You may distribute the
+Executable form of the Covered Software under the terms of this License or under
+the terms of a license of Your choice, which may contain terms different from
+this License, provided that You are in compliance with the terms of this License
+and that the license for the Executable form does not attempt to limit or alter
+the recipients rights in the Source Code form from the rights set forth in this
+License. If You distribute the Covered Software in Executable form under a
+different license, You must make it absolutely clear that any terms which differ
+from this License are offered by You alone, not by the Initial Developer or
+Contributor. You hereby agree to indemnify the Initial Developer and every
+Contributor for any liability incurred by the Initial Developer or such
+Contributor as a result of any such terms You offer.�
+
+3.6. Larger Works. You
+may create a Larger Work by combining Covered Software with other code not
+governed by the terms of this License and distribute the Larger Work as a single
+product. In such a case, You must make sure the requirements of this License are
+fulfilled for the Covered Software.�
+
+4. Versions of the License.�
+
+4.1.
+New Versions. Sun Microsystems, Inc. is the initial license steward and may
+publish revised and/or new versions of this License from time to time. Each
+version will be given a distinguishing version number. Except as provided in
+Section 4.3, no one other than the license steward has the right to modify this
+License.�
+
+4.2. Effect of New Versions. You may always continue to use,
+distribute or otherwise make the Covered Software available under the terms of
+the version of the License under which You originally received the Covered
+Software. If the Initial Developer includes a notice in the Original Software
+prohibiting it from being distributed or otherwise made available under any
+subsequent version of the License, You must distribute and make the Covered
+Software available under the terms of the version of the License under which You
+originally received the Covered Software. Otherwise, You may also choose to use,
+distribute or otherwise make the Covered Software available under the terms of
+any subsequent version of the License published by the license
+steward.�
+
+4.3. Modified Versions. When You are an Initial Developer and You
+want to create a new license for Your Original Software, You may create and use
+a modified version of this License if You: (a)�rename the license and remove
+any references to the name of the license steward (except to note that the
+license differs from this License); and (b)�otherwise make it clear that the
+license contains terms which differ from this License.�
+
+5. DISCLAIMER OF WARRANTY.
+
+COVERED SOFTWARE IS PROVIDED UNDER THIS LICENSE ON AN AS IS BASIS,
+WITHOUT WARRANTY OF ANY KIND, EITHER EXPRESSED OR IMPLIED, INCLUDING, WITHOUT
+LIMITATION, WARRANTIES THAT THE COVERED SOFTWARE IS FREE OF DEFECTS,
+MERCHANTABLE, FIT FOR A PARTICULAR PURPOSE OR NON-INFRINGING. THE ENTIRE RISK AS
+TO THE QUALITY AND PERFORMANCE OF THE COVERED SOFTWARE IS WITH YOU. SHOULD ANY
+COVERED SOFTWARE PROVE DEFECTIVE IN ANY RESPECT, YOU (NOT THE INITIAL DEVELOPER
+OR ANY OTHER CONTRIBUTOR) ASSUME THE COST OF ANY NECESSARY SERVICING, REPAIR OR
+CORRECTION. THIS DISCLAIMER OF WARRANTY CONSTITUTES AN ESSENTIAL PART OF THIS
+LICENSE. NO USE OF ANY COVERED SOFTWARE IS AUTHORIZED HEREUNDER EXCEPT UNDER
+THIS DISCLAIMER.�
+
+6. TERMINATION.�
+
+6.1. This License and the rights
+granted hereunder will terminate automatically if You fail to comply with terms
+herein and fail to cure such breach within 30 days of becoming aware of the
+breach. Provisions which, by their nature, must remain in effect beyond the
+termination of this License shall survive.�
+
+6.2. If You assert a patent
+infringement claim (excluding declaratory judgment actions) against Initial
+Developer or a Contributor (the Initial Developer or Contributor against whom
+You assert such claim is referred to as Participant) alleging that the
+Participant Software (meaning the Contributor Version where the Participant is a
+Contributor or the Original Software where the Participant is the Initial
+Developer) directly or indirectly infringes any patent, then any and all rights
+granted directly or indirectly to You by such Participant, the Initial Developer
+(if the Initial Developer is not the Participant) and all Contributors under
+Sections�2.1 and/or 2.2 of this License shall, upon 60 days notice from
+Participant terminate prospectively and automatically at the expiration of such
+60 day notice period, unless if within such 60 day period You withdraw Your
+claim with respect to the Participant Software against such Participant either
+unilaterally or pursuant to a written agreement with Participant.�
+
+6.3. In
+the event of termination under Sections�6.1 or 6.2 above, all end user licenses
+that have been validly granted by You or any distributor hereunder prior to
+termination (excluding licenses granted to You by any distributor) shall survive
+termination.�
+
+7. LIMITATION OF LIABILITY.
+UNDER NO CIRCUMSTANCES AND UNDER
+NO LEGAL THEORY, WHETHER TORT (INCLUDING NEGLIGENCE), CONTRACT, OR OTHERWISE,
+SHALL YOU, THE INITIAL DEVELOPER, ANY OTHER CONTRIBUTOR, OR ANY DISTRIBUTOR OF
+COVERED SOFTWARE, OR ANY SUPPLIER OF ANY OF SUCH PARTIES, BE LIABLE TO ANY
+PERSON FOR ANY INDIRECT, SPECIAL, INCIDENTAL, OR CONSEQUENTIAL DAMAGES OF ANY
+CHARACTER INCLUDING, WITHOUT LIMITATION, DAMAGES FOR LOST PROFITS, LOSS OF
+GOODWILL, WORK STOPPAGE, COMPUTER FAILURE OR MALFUNCTION, OR ANY AND ALL OTHER
+COMMERCIAL DAMAGES OR LOSSES, EVEN IF SUCH PARTY SHALL HAVE BEEN INFORMED OF THE
+POSSIBILITY OF SUCH DAMAGES. THIS LIMITATION OF LIABILITY SHALL NOT APPLY TO
+LIABILITY FOR DEATH OR PERSONAL INJURY RESULTING FROM SUCH PARTYS NEGLIGENCE TO
+THE EXTENT APPLICABLE LAW PROHIBITS SUCH LIMITATION. SOME JURISDICTIONS DO NOT
+ALLOW THE EXCLUSION OR LIMITATION OF INCIDENTAL OR CONSEQUENTIAL DAMAGES, SO
+THIS EXCLUSION AND LIMITATION MAY NOT APPLY TO YOU.�
+
+8. U.S. GOVERNMENT END USERS.
+
+The Covered Software is a commercial item, as that term is defined in
+48�C.F.R.�2.101 (Oct. 1995), consisting of commercial computer software (as
+that term is defined at 48 C.F.R. �252.227-7014(a)(1)) and commercial computer
+software documentation as such terms are used in 48�C.F.R.�12.212 (Sept.
+1995). Consistent with 48 C.F.R. 12.212 and 48 C.F.R. 227.7202-1 through
+227.7202-4 (June 1995), all U.S. Government End Users acquire Covered Software
+with only those rights set forth herein. This U.S. Government Rights clause is
+in lieu of, and supersedes, any other FAR, DFAR, or other clause or provision
+that addresses Government rights in computer software under this
+License.�
+
+9. MISCELLANEOUS.
+This License represents the complete agreement
+concerning subject matter hereof. If any provision of this License is held to be
+unenforceable, such provision shall be reformed only to the extent necessary to
+make it enforceable. This License shall be governed by the law of the
+jurisdiction specified in a notice contained within the Original Software
+(except to the extent applicable law, if any, provides otherwise), excluding
+such jurisdictions conflict-of-law provisions. Any litigation relating to this
+License shall be subject to the jurisdiction of the courts located in the
+jurisdiction and venue specified in a notice contained within the Original
+Software, with the losing party responsible for costs, including, without
+limitation, court costs and reasonable attorneys fees and expenses. The
+application of the United Nations Convention on Contracts for the International
+Sale of Goods is expressly excluded. Any law or regulation which provides that
+the language of a contract shall be construed against the drafter shall not
+apply to this License. You agree that You alone are responsible for compliance
+with the United States export administration regulations (and the export control
+laws and regulation of any other countries) when You use, distribute or
+otherwise make available any Covered Software.�
+
+10. RESPONSIBILITY FOR CLAIMS.
+As between Initial Developer and the Contributors, each party is
+responsible for claims and damages arising, directly or indirectly, out of its
+utilization of rights under this License and You agree to work with Initial
+Developer and Contributors to distribute such responsibility on an equitable
+basis. Nothing herein is intended or shall be deemed to constitute any admission
+of liability.�
+
+The binary distribution of this product bundles these dependencies under the
+following license:
+Jersey 1.9
+JAXB API bundle for GlassFish V3 2.2.2
+JAXB RI 2.2.3
+--------------------------------------------------------------------------------
+COMMON DEVELOPMENT AND DISTRIBUTION LICENSE (CDDL)Version 1.1
+
+1. Definitions.
+
+1.1. \u201cContributor\u201d means each individual or entity that creates or
+contributes to the creation of Modifications.
+1.2. \u201cContributor Version\u201d means the combination of the Original Software,
+prior Modifications used by a Contributor (if any), and the Modifications made
+by that particular Contributor.
+1.3. \u201cCovered Software\u201d means (a) the Original Software, or (b)
+Modifications, or (c) the combination of files containing Original Software with
+files containing Modifications, in each case including portions thereof.
+1.4. \u201cExecutable\u201d means the Covered Software in any form other than Source
+Code.
+1.5. \u201cInitial Developer\u201d means the individual or entity that first makes
+Original Software available under this License.
+1.6. \u201cLarger Work\u201d means a work which combines Covered Software or portions
+thereof with code not governed by the terms of this License.
+1.7. \u201cLicense\u201d means this document.
+1.8. \u201cLicensable\u201d means having the right to grant, to the maximum extent
+possible, whether at the time of the initial grant or subsequently acquired, any
+and all of the rights conveyed herein.
+1.9. \u201cModifications\u201d means the Source Code and Executable form of any of the
+following:
+A. Any file that results from an addition to, deletion from or modification of
+the contents of a file containing Original Software or previous Modifications;
+B. Any new file that contains any part of the Original Software or previous
+Modification; or
+C. Any new file that is contributed or otherwise made available under the terms
+of this License.
+1.10. \u201cOriginal Software\u201d means the Source Code and Executable form of
+computer software code that is originally released under this License.
+1.11. \u201cPatent Claims\u201d means any patent claim(s), now owned or hereafter
+acquired, including without limitation, method, process, and apparatus claims,
+in any patent Licensable by grantor.
+1.12. \u201cSource Code\u201d means (a) the common form of computer software code in
+which modifications are made and (b) associated documentation included in or
+with such code.
+1.13. \u201cYou\u201d (or \u201cYour\u201d) means an individual or a legal entity exercising
+rights under, and complying with all of the terms of, this License. For legal
+entities, \u201cYou\u201d includes any entity which controls, is controlled by, or is
+under common control with You. For purposes of this definition, \u201ccontrol\u201d
+means (a) the power, direct or indirect, to cause the direction or management of
+such entity, whether by contract or otherwise, or (b) ownership of more than
+fifty percent (50%) of the outstanding shares or beneficial ownership of such
+entity.
+
+2. License Grants.
+
+2.1. The Initial Developer Grant.
+
+Conditioned upon Your compliance with Section 3.1 below and subject to
+third party intellectual property claims, the Initial Developer hereby grants
+You a world-wide, royalty-free, non-exclusive license:
+(a) under intellectual
+property rights (other than patent or trademark) Licensable by Initial
+Developer, to use, reproduce, modify, display, perform, sublicense and
+distribute the Original Software (or portions thereof), with or without
+Modifications, and/or as part of a Larger Work; and
+(b) under Patent Claims
+infringed by the making, using or selling of Original Software, to make, have
+made, use, practice, sell, and offer for sale, and/or otherwise dispose of the
+Original Software (or portions thereof).
+(c) The licenses granted in Sections
+2.1(a) and (b) are effective on the date Initial Developer first distributes or
+otherwise makes the Original Software available to a third party under the terms
+of this License.
+(d) Notwithstanding Section 2.1(b) above, no patent license is
+granted: (1) for code that You delete from the Original Software, or (2) for
+infringements caused by: (i) the modification of the Original Software, or (ii)
+the combination of the Original Software with other software or devices.
+
+2.2. Contributor Grant.
+
+Conditioned upon Your compliance with Section 3.1 below and
+subject to third party intellectual property claims, each Contributor hereby
+grants You a world-wide, royalty-free, non-exclusive license:
+(a) under
+intellectual property rights (other than patent or trademark) Licensable by
+Contributor to use, reproduce, modify, display, perform, sublicense and
+distribute the Modifications created by such Contributor (or portions thereof),
+either on an unmodified basis, with other Modifications, as Covered Software
+and/or as part of a Larger Work; and
+(b) under Patent Claims infringed by the
+making, using, or selling of Modifications made by that Contributor either alone
+and/or in combination with its Contributor Version (or portions of such
+combination), to make, use, sell, offer for sale, have made, and/or otherwise
+dispose of: (1) Modifications made by that Contributor (or portions thereof);
+and (2) the combination of Modifications made by that Contributor with its
+Contributor Version (or portions of such combination).
+(c) The licenses granted
+in Sections 2.2(a) and 2.2(b) are effective on the date Contributor first
+distributes or otherwise makes the Modifications available to a third
+party.
+(d) Notwithstanding Section 2.2(b) above, no patent license is granted:
+(1) for any code that Contributor has deleted from the Contributor Version; (2)
+for infringements caused by: (i) third party modifications of Contributor
+Version, or (ii) the combination of Modifications made by that Contributor with
+other software (except as part of the Contributor Version) or other devices; or
+(3) under Patent Claims infringed by Covered Software in the absence of
+Modifications made by that Contributor.
+
+3. Distribution Obligations.
+
+3.1. Availability of Source Code.
+Any Covered Software that You distribute or
+otherwise make available in Executable form must also be made available in
+Source Code form and that Source Code form must be distributed only under the
+terms of this License. You must include a copy of this License with every copy
+of the Source Code form of the Covered Software You distribute or otherwise make
+available. You must inform recipients of any such Covered Software in Executable
+form as to how they can obtain such Covered Software in Source Code form in a
+reasonable manner on or through a medium customarily used for software
+exchange.
+3.2. Modifications.
+The Modifications that You create or to which
+You contribute are governed by the terms of this License. You represent that You
+believe Your Modifications are Your original creation(s) and/or You have
+sufficient rights to grant the rights conveyed by this License.
+3.3. Required Notices.
+You must include a notice in each of Your Modifications that
+identifies You as the Contributor of the Modification. You may not remove or
+alter any copyright, patent or trademark notices contained within the Covered
+Software, or any notices of licensing or any descriptive text giving attribution
+to any Contributor or the Initial Developer.
+3.4. Application of Additional Terms.
+You may not offer or impose any terms on any Covered Software in Source
+Code form that alters or restricts the applicable version of this License or the
+recipients' rights hereunder. You may choose to offer, and to charge a fee for,
+warranty, support, indemnity or liability obligations to one or more recipients
+of Covered Software. However, you may do so only on Your own behalf, and not on
+behalf of the Initial Developer or any Contributor. You must make it absolutely
+clear that any such warranty, support, indemnity or liability obligation is
+offered by You alone, and You hereby agree to indemnify the Initial Developer
+and every Contributor for any liability incurred by the Initial Developer or
+such Contributor as a result of warranty, support, indemnity or liability terms
+You offer.
+3.5. Distribution of Executable Versions.
+You may distribute the
+Executable form of the Covered Software under the terms of this License or under
+the terms of a license of Your choice, which may contain terms different from
+this License, provided that You are in compliance with the terms of this License
+and that the license for the Executable form does not attempt to limit or alter
+the recipient's rights in the Source Code form from the rights set forth in
+this License. If You distribute the Covered Software in Executable form under a
+different license, You must make it absolutely clear that any terms which differ
+from this License are offered by You alone, not by the Initial Developer or
+Contributor. You hereby agree to indemnify the Initial Developer and every
+Contributor for any liability incurred by the Initial Developer or such
+Contributor as a result of any such terms You offer.
+3.6. Larger Works.
+You
+may create a Larger Work by combining Covered Software with other code not
+governed by the terms of this License and distribute the Larger Work as a single
+product. In such a case, You must make sure the requirements of this License are
+fulfilled for the Covered Software.
+
+4. Versions of the License.
+
+4.1. New Versions.
+Oracle is the initial license steward and may publish revised and/or
+new versions of this License from time to time. Each version will be given a
+distinguishing version number. Except as provided in Section 4.3, no one other
+than the license steward has the right to modify this License.
+4.2. Effect of New Versions.
+You may always continue to use, distribute or otherwise make the
+Covered Software available under the terms of the version of the License under
+which You originally received the Covered Software. If the Initial Developer
+includes a notice in the Original Software prohibiting it from being distributed
+or otherwise made available under any subsequent version of the License, You
+must distribute and make the Covered Software available under the terms of the
+version of the License under which You originally received the Covered Software.
+Otherwise, You may also choose to use, distribute or otherwise make the Covered
+Software available under the terms of any subsequent version of the License
+published by the license steward.
+4.3. Modified Versions.
+When You are an
+Initial Developer and You want to create a new license for Your Original
+Software, You may create and use a modified version of this License if You: (a)
+rename the license and remove any references to the name of the license steward
+(except to note that the license differs from this License); and (b) otherwise
+make it clear that the license contains terms which differ from this
+License.
+
+5. DISCLAIMER OF WARRANTY.
+
+COVERED SOFTWARE IS PROVIDED UNDER THIS
+LICENSE ON AN \u201cAS IS\u201d BASIS, WITHOUT WARRANTY OF ANY KIND, EITHER EXPRESSED
+OR IMPLIED, INCLUDING, WITHOUT LIMITATION, WARRANTIES THAT THE COVERED SOFTWARE
+IS FREE OF DEFECTS, MERCHANTABLE, FIT FOR A PARTICULAR PURPOSE OR
+NON-INFRINGING. THE ENTIRE RISK AS TO THE QUALITY AND PERFORMANCE OF THE COVERED
+SOFTWARE IS WITH YOU. SHOULD ANY COVERED SOFTWARE PROVE DEFECTIVE IN ANY
+RESPECT, YOU (NOT THE INITIAL DEVELOPER OR ANY OTHER CONTRIBUTOR) ASSUME THE
+COST OF ANY NECESSARY SERVICING, REPAIR OR CORRECTION. THIS DISCLAIMER OF
+WARRANTY CONSTITUTES AN ESSENTIAL PART OF THIS LICENSE. NO USE OF ANY COVERED
+SOFTWARE IS AUTHORIZED HEREUNDER EXCEPT UNDER THIS DISCLAIMER.
+
+6. TERMINATION.
+
+6.1. This License and the rights granted hereunder will
+terminate automatically if You fail to comply with terms herein and fail to cure
+such breach within 30 days of becoming aware of the breach. Provisions which, by
+their nature, must remain in effect beyond the termination of this License shall
+survive.
+6.2. If You assert a patent infringement claim (excluding declaratory
+judgment actions) against Initial Developer or a Contributor (the Initial
+Developer or Contributor against whom You assert such claim is referred to as
+\u201cParticipant\u201d) alleging that the Participant Software (meaning the
+Contributor Version where the Participant is a Contributor or the Original
+Software where the Participant is the Initial Developer) directly or indirectly
+infringes any patent, then any and all rights granted directly or indirectly to
+You by such Participant, the Initial Developer (if the Initial Developer is not
+the Participant) and all Contributors under Sections 2.1 and/or 2.2 of this
+License shall, upon 60 days notice from Participant terminate prospectively and
+automatically at the expiration of such 60 day notice period, unless if within
+such 60 day period You withdraw Your claim with respect to the Participant
+Software against such Participant either unilaterally or pursuant to a written
+agreement with Participant.
+6.3. If You assert a patent infringement claim
+against Participant alleging that the Participant Software directly or
+indirectly infringes any patent where such claim is resolved (such as by license
+or settlement) prior to the initiation of patent infringement litigation, then
+the reasonable value of the licenses granted by such Participant under Sections
+2.1 or 2.2 shall be taken into account in determining the amount or value of any
+payment or license.
+6.4. In the event of termination under Sections 6.1 or 6.2
+above, all end user licenses that have been validly granted by You or any
+distributor hereunder prior to termination (excluding licenses granted to You by
+any distributor) shall survive termination.
+
+7. LIMITATION OF LIABILITY.
+
+UNDER NO CIRCUMSTANCES AND UNDER NO LEGAL THEORY, WHETHER TORT
+(INCLUDING NEGLIGENCE), CONTRACT, OR OTHERWISE, SHALL YOU, THE INITIAL
+DEVELOPER, ANY OTHER CONTRIBUTOR, OR ANY DISTRIBUTOR OF COVERED SOFTWARE, OR ANY
+SUPPLIER OF ANY OF SUCH PARTIES, BE LIABLE TO ANY PERSON FOR ANY INDIRECT,
+SPECIAL, INCIDENTAL, OR CONSEQUENTIAL DAMAGES OF ANY CHARACTER INCLUDING,
+WITHOUT LIMITATION, DAMAGES FOR LOSS OF GOODWILL, WORK STOPPAGE, COMPUTER
+FAILURE OR MALFUNCTION, OR ANY AND ALL OTHER COMMERCIAL DAMAGES OR LOSSES, EVEN
+IF SUCH PARTY SHALL HAVE BEEN INFORMED OF THE POSSIBILITY OF SUCH DAMAGES. THIS
+LIMITATION OF LIABILITY SHALL NOT APPLY TO LIABILITY FOR DEATH OR PERSONAL
+INJURY RESULTING FROM SUCH PARTY'S NEGLIGENCE TO THE EXTENT APPLICABLE LAW
+PROHIBITS SUCH LIMITATION. SOME JURISDICTIONS DO NOT ALLOW THE EXCLUSION OR
+LIMITATION OF INCIDENTAL OR CONSEQUENTIAL DAMAGES, SO THIS EXCLUSION AND
+LIMITATION MAY NOT APPLY TO YOU.
+
+8. U.S. GOVERNMENT END USERS.
+
+The Covered
+Software is a \u201ccommercial item,\u201d as that term is defined in 48 C.F.R. 2.101
+(Oct. 1995), consisting of \u201ccommercial computer software\u201d (as that term is
+defined at 48 C.F.R. � 252.227-7014(a)(1)) and \u201ccommercial computer software
+documentation\u201d as such terms are used in 48 C.F.R. 12.212 (Sept. 1995).
+Consistent with 48 C.F.R. 12.212 and 48 C.F.R. 227.7202-1 through 227.7202-4
+(June 1995), all U.S. Government End Users acquire Covered Software with only
+those rights set forth herein. This U.S. Government Rights clause is in lieu of,
+and supersedes, any other FAR, DFAR, or other clause or provision that addresses
+Government rights in computer software under this License.
+
+9. MISCELLANEOUS.
+
+This License represents the complete agreement concerning
+subject matter hereof. If any provision of this License is held to be
+unenforceable, such provision shall be reformed only to the extent necessary to
+make it enforceable. This License shall be governed by the law of the
+jurisdiction specified in a notice contained within the Original Software
+(except to the extent applicable law, if any, provides otherwise), excluding
+such jurisdiction's conflict-of-law provisions. Any litigation relating to this
+License shall be subject to the jurisdiction of the courts located in the
+jurisdiction and venue specified in a notice contained within the Original
+Software, with the losing party responsible for costs, including, without
+limitation, court costs and reasonable attorneys' fees and expenses. The
+application of the United Nations Convention on Contracts for the International
+Sale of Goods is expressly excluded. Any law or regulation which provides that
+the language of a contract shall be construed against the drafter shall not
+apply to this License. You agree that You alone are responsible for compliance
+with the United States export administration regulations (and the export control
+laws and regulation of any other countries) when You use, distribute or
+otherwise make available any Covered Software.
+
+10. RESPONSIBILITY FOR CLAIMS.
+
+As between Initial Developer and the Contributors, each party is
+responsible for claims and damages arising, directly or indirectly, out of its
+utilization of rights under this License and You agree to work with Initial
+Developer and Contributors to distribute such responsibility on an equitable
+basis. Nothing herein is intended or shall be deemed to constitute any admission
+of liability.
+
+The binary distribution of this product bundles these dependencies under the
+following license:
+Protocol Buffer Java API 2.5.0
+--------------------------------------------------------------------------------
+This license applies to all parts of Protocol Buffers except the following:
+
+  - Atomicops support for generic gcc, located in
+    src/google/protobuf/stubs/atomicops_internals_generic_gcc.h.
+    This file is copyrighted by Red Hat Inc.
+
+  - Atomicops support for AIX/POWER, located in
+    src/google/protobuf/stubs/atomicops_internals_power.h.
+    This file is copyrighted by Bloomberg Finance LP.
+
+Copyright 2014, Google Inc.  All rights reserved.
+
+Redistribution and use in source and binary forms, with or without
+modification, are permitted provided that the following conditions are
+met:
+
+    * Redistributions of source code must retain the above copyright
+notice, this list of conditions and the following disclaimer.
+    * Redistributions in binary form must reproduce the above
+copyright notice, this list of conditions and the following disclaimer
+in the documentation and/or other materials provided with the
+distribution.
+    * Neither the name of Google Inc. nor the names of its
+contributors may be used to endorse or promote products derived from
+this software without specific prior written permission.
+
+THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+"AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
+A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
+OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
+LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
+THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
+OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+
+Code generated by the Protocol Buffer compiler is owned by the owner
+of the input file used when generating it.  This code is not
+standalone and requires a support library to be linked with it.  This
+support library is itself covered by the above license.
+
+For:
+XML Commons External Components XML APIs 1.3.04
+--------------------------------------------------------------------------------
+By obtaining, using and/or copying this work, you (the licensee) agree that you
+have read, understood, and will comply with the following terms and conditions.
+
+Permission to copy, modify, and distribute this software and its documentation,
+with or without modification, for any purpose and without fee or royalty is
+hereby granted, provided that you include the following on ALL copies of the
+software and documentation or portions thereof, including modifications:
+- The full text of this NOTICE in a location viewable to users of the
+redistributed or derivative work.
+- Any pre-existing intellectual property disclaimers, notices, or terms and
+conditions. If none exist, the W3C Software Short Notice should be included
+(hypertext is preferred, text is permitted) within the body of any redistributed
+or derivative code.
+- Notice of any changes or modifications to the files, including the date changes
+were made. (We recommend you provide URIs to the location from which the code is
+derived.)
+
+The binary distribution of this product bundles these dependencies under the
+following license:
+JUnit 4.11
+ecj-4.3.1.jar
+--------------------------------------------------------------------------------
+Eclipse Public License - v 1.0
+
+THE ACCOMPANYING PROGRAM IS PROVIDED UNDER THE TERMS OF THIS ECLIPSE PUBLIC
+LICENSE ("AGREEMENT"). ANY USE, REPRODUCTION OR DISTRIBUTION OF THE PROGRAM
+CONSTITUTES RECIPIENT'S ACCEPTANCE OF THIS AGREEMENT.
+
+1. DEFINITIONS
+
+"Contribution" means:
+
+a) in the case of the initial Contributor, the initial code and documentation
+distributed under this Agreement, and
+b) in the case of each subsequent Contributor:
+i) changes to the Program, and
+ii) additions to the Program;
+where such changes and/or additions to the Program originate from and are
+distributed by that particular Contributor. A Contribution 'originates' from a
+Contributor if it was added to the Program by such Contributor itself or anyone
+acting on such Contributor's behalf. Contributions do not include additions to
+the Program which: (i) are separate modules of software distributed in
+conjunction with the Program under their own license agreement, and (ii) are not
+derivative works of the Program.
+"Contributor" means any person or entity that distributes the Program.
+
+"Licensed Patents" mean patent claims licensable by a Contributor which are
+necessarily infringed by the use or sale of its Contribution alone or when
+combined with the Program.
+
+"Program" means the Contributions distributed in accordance with this Agreement.
+
+"Recipient" means anyone who receives the Program under this Agreement,
+including all Contributors.
+
+2. GRANT OF RIGHTS
+
+a) Subject to the terms of this Agreement, each Contributor hereby grants
+Recipient a non-exclusive, worldwide, royalty-free copyright license to
+reproduce, prepare derivative works of, publicly display, publicly perform,
+distribute and sublicense the Contribution of such Contributor, if any, and such
+derivative works, in source code and object code form.
+b) Subject to the terms of this Agreement, each Contributor hereby grants
+Recipient a non-exclusive, worldwide, royalty-free patent license under Licensed
+Patents to make, use, sell, offer to sell, import and otherwise transfer the
+Contribution of such Contributor, if any, in source code and object code form.
+This patent license shall apply to the combination of the Contribution and the
+Program if, at the time the Contribution is added by the Contributor, such
+addition of the Contribution causes such combination to be covered by the
+Licensed Patents. The patent license shall not apply to any other combinations
+which include the Contribution. No hardware per se is licensed hereunder.
+c) Recipient understands that although each Contributor grants the licenses to
+its Contributions set forth herein, no assurances are provided by any
+Contributor that the Program does not infringe the patent or other intellectual
+property rights of any other entity. Each Contributor disclaims any liability to
+Recipient for claims brought by any other entity based on infringement of
+intellectual property rights or otherwise. As a condition to exercising the
+rights and licenses granted hereunder, each Recipient hereby assumes sole
+responsibility to secure any other intellectual property rights needed, if any.
+For example, if a third party patent license is required to allow Recipient to
+distribute the Program, it is Recipient's responsibility to acquire that license
+before distributing the Program.
+d) Each Contributor represents that to its knowledge it has sufficient copyright
+rights in its Contribution, if any, to grant the copyright license set forth in
+this Agreement.
+3. REQUIREMENTS
+
+A Contributor may choose to distribute the Program in object code form under its
+own license agreement, provided that:
+
+a) it complies with the terms and conditions of this Agreement; and
+b) its license agreement:
+i) effectively disclaims on behalf of all Contributors all warranties and
+conditions, express and implied, including warranties or conditions of title and
+non-infringement, and implied warranties or conditions of merchantability and
+fitness for a particular purpose;
+ii) effectively excludes on behalf of all Contributors all liability for
+damages, including direct, indirect, special, incidental and consequential
+damages, such as lost profits;
+iii) states that any provisions which differ from this Agreement are offered by
+that Contributor alone and not by any other party; and
+iv) states that source code for the Program is available from such Contributor,
+and informs licensees how to obtain it in a reasonable manner on or through a
+medium customarily used for software exchange.
+When the Program is made available in source code form:
+
+a) it must be made available under this Agreement; and
+b) a copy of this Agreement must be included with each copy of the Program.
+Contributors may not remove or alter any copyright notices contained within the
+Program.
+
+Each Contributor must identify itself as the originator of its Contribution, if
+any, in a manner that reasonably allows subsequent Recipients to identify the
+originator of the Contribution.
+
+4. COMMERCIAL DISTRIBUTION
+
+Commercial distributors of software may accept certain responsibilities with
+respect to end users, business partners and the like. While this license is
+intended to facilitate the commercial use of the Program, the Contributor who
+includes the Program in a commercial product offering should do so in a manner
+which does not create potential liability for other Contributors. Therefore, if
+a Contributor includes the Program in a commercial product offering, such
+Contributor ("Commercial Contributor") hereby agrees to defend and indemnify
+every other Contributor ("Indemnified Contributor") against any losses, damages
+and costs (collectively "Losses") arising from claims, lawsuits and other legal
+actions brought by a third party against the Indemnified Contributor to the
+extent caused by the acts or omissions of such Commercial Contributor in
+connection with its distribution of the Program in a commercial product
+offering. The obligations in this section do not apply to any claims or Losses
+relating to any actual or alleged intellectual property infringement. In order
+to qualify, an Indemnified Contributor must: a) promptly notify the Commercial
+Contributor in writing of such claim, and b) allow the Commercial Contributor to
+control, and cooperate with the Commercial Contributor in, the defense and any
+related settlement negotiations. The Indemnified Contributor may participate in
+any such claim at its own expense.
+
+For example, a Contributor might include the Program in a commercial product
+offering, Product X. That Contributor is then a Commercial Contributor. If that
+Commercial Contributor then makes performance claims, or offers warranties
+related to Product X, those performance claims and warranties are such
+Commercial Contributor's responsibility alone. Under this section, the
+Commercial Contributor would have to defend claims against the other
+Contributors related to those performance claims and warranties, and if a court
+requires any other Contributor to pay any damages as a result, the Commercial
+Contributor must pay those damages.
+
+5. NO WARRANTY
+
+EXCEPT AS EXPRESSLY SET FORTH IN THIS AGREEMENT, THE PROGRAM IS PROVIDED ON AN
+"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, EITHER EXPRESS OR
+IMPLIED INCLUDING, WITHOUT LIMITATION, ANY WARRANTIES OR CONDITIONS OF TITLE,
+NON-INFRINGEMENT, MERCHANTABILITY OR FITNESS FOR A PARTICULAR PURPOSE. Each
+Recipient is solely responsible for determining the appropriateness of using and
+distributing the Program and assumes all risks associated with its exercise of
+rights under this Agreement , including but not limited to the risks and costs
+of program errors, compliance with applicable laws, damage to or loss of data,
+programs or equipment, and unavailability or interruption of operations.
+
+6. DISCLAIMER OF LIABILITY
+
+EXCEPT AS EXPRESSLY SET FORTH IN THIS AGREEMENT, NEITHER RECIPIENT NOR ANY
+CONTRIBUTORS SHALL HAVE ANY LIABILITY FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING WITHOUT LIMITATION LOST
+PROFITS), HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT,
+STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY
+OUT OF THE USE OR DISTRIBUTION OF THE PROGRAM OR THE EXERCISE OF ANY RIGHTS
+GRANTED HEREUNDER, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGES.
+
+7. GENERAL
+
+If any provision of this Agreement is invalid or unenforceable under applicable
+law, it shall not affect the validity or enforceability of the remainder of the
+terms of this Agreement, and without further action by the parties hereto, such
+provision shall be reformed to the minimum extent necessary to make such
+provision valid and enforceable.
+
+If Recipient institutes patent litigation against any entity (including a
+cross-claim or counterclaim in a lawsuit) alleging that the Program itself
+(excluding combinations of the Program with other software or hardware)
+infringes such Recipient's patent(s), then such Recipient's rights granted under
+Section 2(b) shall terminate as of the date such litigation is filed.
+
+All Recipient's rights under this Agreement shall terminate if it fails to
+comply with any of the material terms or conditions of this Agreement and does
+not cure such failure in a reasonable period of time after becoming aware of
+such noncompliance. If all Recipient's rights under this Agreement terminate,
+Recipient agrees to cease use and distribution of the Program as soon as
+reasonably practicable. However, Recipient's obligations under this Agreement
+and any licenses granted by Recipient relating to the Program shall continue and
+survive.
+
+Everyone is permitted to copy and distribute copies of this Agreement, but in
+order to avoid inconsistency the Agreement is copyrighted and may only be
+modified in the following manner. The Agreement Steward reserves the right to
+publish new versions (including revisions) of this Agreement from time to time.
+No one other than the Agreement Steward has the right to modify this Agreement.
+The Eclipse Foundation is the initial Agreement Steward. The Eclipse Foundation
+may assign the responsibility to serve as the Agreement Steward to a suitable
+separate entity. Each new version of the Agreement will be given a
+distinguishing version number. The Program (including Contributions) may always
+be distributed subject to the version of the Agreement under which it was
+received. In addition, after a new version of the Agreement is published,
+Contributor may elect to distribute the Program (including its Contributions)
+under the new version. Except as expressly stated in Sections 2(a) and 2(b)
+above, Recipient receives no rights or licenses to the intellectual property of
+any Contributor under this Agreement, whether expressly, by implication,
+estoppel or otherwise. All rights in the Program not expressly granted under
+this Agreement are reserved.
+
+This Agreement is governed by the laws of the State of New York and the
+intellectual property laws of the United States of America. No party to this
+Agreement will bring a legal action under this Agreement more than one year
+after the cause of action arose. Each party waives its rights to a jury trial in
+any resulting litigation.
+
+The binary distribution of this product bundles these dependencies under the
+following license:
+ASM Core 3.2
+JSch 0.1.51
+ParaNamer Core 2.3
+JLine 0.9.94
+leveldbjni-all 1.8
+Hamcrest Core 1.3
+xmlenc Library 0.52
+--------------------------------------------------------------------------------
+Redistribution and use in source and binary forms, with or without
+modification, are permitted provided that the following conditions are met:
+    * Redistributions of source code must retain the above copyright
+      notice, this list of conditions and the following disclaimer.
+    * Redistributions in binary form must reproduce the above copyright
+      notice, this list of conditions and the following disclaimer in the
+      documentation and/or other materials provided with the distribution.
+    * Neither the name of the <organization> nor the
+      names of its contributors may be used to endorse or promote products
+      derived from this software without specific prior written permission.
+
+THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" AND
+ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED
+WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE
+DISCLAIMED. IN NO EVENT SHALL <COPYRIGHT HOLDER> BE LIABLE FOR ANY
+DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES
+(INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES;
+LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND
+ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS
+SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+
+The binary distribution of this product bundles these dependencies under the
+following license:
+FindBugs-jsr305 3.0.0
+--------------------------------------------------------------------------------
+Redistribution and use in source and binary forms, with or without
+modification, are permitted provided that the following conditions are met:
+
+1. Redistributions of source code must retain the above copyright notice, this
+   list of conditions and the following disclaimer.
+2. Redistributions in binary form must reproduce the above copyright notice,
+   this list of conditions and the following disclaimer in the documentation
+   and/or other materials provided with the distribution.
+
+THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" AND
+ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED
+WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE
+DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE LIABLE FOR
+ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES
+(INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES;
+LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND
+ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS
+SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+
+The views and conclusions contained in the software and documentation are those
+of the authors and should not be interpreted as representing official policies,
+either expressed or implied, of the FreeBSD Project.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/098ae11e/NOTICE.txt
----------------------------------------------------------------------
diff --git a/NOTICE.txt b/NOTICE.txt
index efa865f..0c729e8 100644
--- a/NOTICE.txt
+++ b/NOTICE.txt
@@ -15,3 +15,269 @@ which has the following notices:
 * This product includes software developed at
   Progress Software Corporation and/or its  subsidiaries or affiliates.
 * This product includes software developed by IBM Corporation and others.
+
+The binary distribution of this product bundles binaries of
+AWS Java SDK 1.10.6,
+which has the following notices:
+ * This software includes third party software subject to the following
+ copyrights: - XML parsing and utility functions from JetS3t - Copyright
+ 2006-2009 James Murty. - JSON parsing and utility functions from JSON.org -
+ Copyright 2002 JSON.org. - PKCS#1 PEM encoded private key parsing and utility
+ functions from oauth.googlecode.com - Copyright 1998-2010 AOL Inc.
+
+The binary distribution of this product bundles binaries of
+Gson 2.2.4,
+which has the following notices:
+
+                            The Netty Project
+                            =================
+
+Please visit the Netty web site for more information:
+
+  * http://netty.io/
+
+Copyright 2014 The Netty Project
+
+The Netty Project 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.
+
+Also, please refer to each LICENSE.<component>.txt file, which is located in
+the 'license' directory of the distribution file, for the license terms of the
+components that this product depends on.
+
+-------------------------------------------------------------------------------
+This product contains the extensions to Java Collections Framework which has
+been derived from the works by JSR-166 EG, Doug Lea, and Jason T. Greene:
+
+  * LICENSE:
+    * license/LICENSE.jsr166y.txt (Public Domain)
+  * HOMEPAGE:
+    * http://gee.cs.oswego.edu/cgi-bin/viewcvs.cgi/jsr166/
+    * http://viewvc.jboss.org/cgi-bin/viewvc.cgi/jbosscache/experimental/jsr166/
+
+This product contains a modified version of Robert Harder's Public Domain
+Base64 Encoder and Decoder, which can be obtained at:
+
+  * LICENSE:
+    * license/LICENSE.base64.txt (Public Domain)
+  * HOMEPAGE:
+    * http://iharder.sourceforge.net/current/java/base64/
+
+This product contains a modified portion of 'Webbit', an event based
+WebSocket and HTTP server, which can be obtained at:
+
+  * LICENSE:
+    * license/LICENSE.webbit.txt (BSD License)
+  * HOMEPAGE:
+    * https://github.com/joewalnes/webbit
+
+This product contains a modified portion of 'SLF4J', a simple logging
+facade for Java, which can be obtained at:
+
+  * LICENSE:
+    * license/LICENSE.slf4j.txt (MIT License)
+  * HOMEPAGE:
+    * http://www.slf4j.org/
+
+This product contains a modified portion of 'ArrayDeque', written by Josh
+Bloch of Google, Inc:
+
+  * LICENSE:
+    * license/LICENSE.deque.txt (Public Domain)
+
+This product contains a modified portion of 'Apache Harmony', an open source
+Java SE, which can be obtained at:
+
+  * LICENSE:
+    * license/LICENSE.harmony.txt (Apache License 2.0)
+  * HOMEPAGE:
+    * http://archive.apache.org/dist/harmony/
+
+This product contains a modified version of Roland Kuhn's ASL2
+AbstractNodeQueue, which is based on Dmitriy Vyukov's non-intrusive MPSC queue.
+It can be obtained at:
+
+  * LICENSE:
+    * license/LICENSE.abstractnodequeue.txt (Public Domain)
+  * HOMEPAGE:
+    * https://github.com/akka/akka/blob/wip-2.2.3-for-scala-2.11/akka-actor/src/main/java/akka/dispatch/AbstractNodeQueue.java
+
+This product contains a modified portion of 'jbzip2', a Java bzip2 compression
+and decompression library written by Matthew J. Francis. It can be obtained at:
+
+  * LICENSE:
+    * license/LICENSE.jbzip2.txt (MIT License)
+  * HOMEPAGE:
+    * https://code.google.com/p/jbzip2/
+
+This product contains a modified portion of 'libdivsufsort', a C API library to construct
+the suffix array and the Burrows-Wheeler transformed string for any input string of
+a constant-size alphabet written by Yuta Mori. It can be obtained at:
+
+  * LICENSE:
+    * license/LICENSE.libdivsufsort.txt (MIT License)
+  * HOMEPAGE:
+    * https://code.google.com/p/libdivsufsort/
+
+This product contains a modified portion of Nitsan Wakart's 'JCTools', Java Concurrency Tools for the JVM,
+ which can be obtained at:
+
+  * LICENSE:
+    * license/LICENSE.jctools.txt (ASL2 License)
+  * HOMEPAGE:
+    * https://github.com/JCTools/JCTools
+
+This product optionally depends on 'JZlib', a re-implementation of zlib in
+pure Java, which can be obtained at:
+
+  * LICENSE:
+    * license/LICENSE.jzlib.txt (BSD style License)
+  * HOMEPAGE:
+    * http://www.jcraft.com/jzlib/
+
+This product optionally depends on 'Compress-LZF', a Java library for encoding and
+decoding data in LZF format, written by Tatu Saloranta. It can be obtained at:
+
+  * LICENSE:
+    * license/LICENSE.compress-lzf.txt (Apache License 2.0)
+  * HOMEPAGE:
+    * https://github.com/ning/compress
+
+This product optionally depends on 'lz4', a LZ4 Java compression
+and decompression library written by Adrien Grand. It can be obtained at:
+
+  * LICENSE:
+    * license/LICENSE.lz4.txt (Apache License 2.0)
+  * HOMEPAGE:
+    * https://github.com/jpountz/lz4-java
+
+This product optionally depends on 'lzma-java', a LZMA Java compression
+and decompression library, which can be obtained at:
+
+  * LICENSE:
+    * license/LICENSE.lzma-java.txt (Apache License 2.0)
+  * HOMEPAGE:
+    * https://github.com/jponge/lzma-java
+
+This product contains a modified portion of 'jfastlz', a Java port of FastLZ compression
+and decompression library written by William Kinney. It can be obtained at:
+
+  * LICENSE:
+    * license/LICENSE.jfastlz.txt (MIT License)
+  * HOMEPAGE:
+    * https://code.google.com/p/jfastlz/
+
+This product contains a modified portion of and optionally depends on 'Protocol Buffers', Google's data
+interchange format, which can be obtained at:
+
+  * LICENSE:
+    * license/LICENSE.protobuf.txt (New BSD License)
+  * HOMEPAGE:
+    * http://code.google.com/p/protobuf/
+
+This product optionally depends on 'Bouncy Castle Crypto APIs' to generate
+a temporary self-signed X.509 certificate when the JVM does not provide the
+equivalent functionality.  It can be obtained at:
+
+  * LICENSE:
+    * license/LICENSE.bouncycastle.txt (MIT License)
+  * HOMEPAGE:
+    * http://www.bouncycastle.org/
+
+This product optionally depends on 'Snappy', a compression library produced
+by Google Inc, which can be obtained at:
+
+  * LICENSE:
+    * license/LICENSE.snappy.txt (New BSD License)
+  * HOMEPAGE:
+    * http://code.google.com/p/snappy/
+
+This product optionally depends on 'JBoss Marshalling', an alternative Java
+serialization API, which can be obtained at:
+
+  * LICENSE:
+    * license/LICENSE.jboss-marshalling.txt (GNU LGPL 2.1)
+  * HOMEPAGE:
+    * http://www.jboss.org/jbossmarshalling
+
+This product optionally depends on 'Caliper', Google's micro-
+benchmarking framework, which can be obtained at:
+
+  * LICENSE:
+    * license/LICENSE.caliper.txt (Apache License 2.0)
+  * HOMEPAGE:
+    * http://code.google.com/p/caliper/
+
+This product optionally depends on 'Apache Commons Logging', a logging
+framework, which can be obtained at:
+
+  * LICENSE:
+    * license/LICENSE.commons-logging.txt (Apache License 2.0)
+  * HOMEPAGE:
+    * http://commons.apache.org/logging/
+
+This product optionally depends on 'Apache Log4J', a logging framework, which
+can be obtained at:
+
+  * LICENSE:
+    * license/LICENSE.log4j.txt (Apache License 2.0)
+  * HOMEPAGE:
+    * http://logging.apache.org/log4j/
+
+This product optionally depends on 'Aalto XML', an ultra-high performance
+non-blocking XML processor, which can be obtained at:
+
+  * LICENSE:
+    * license/LICENSE.aalto-xml.txt (Apache License 2.0)
+  * HOMEPAGE:
+    * http://wiki.fasterxml.com/AaltoHome
+
+This product contains a modified version of 'HPACK', a Java implementation of
+the HTTP/2 HPACK algorithm written by Twitter. It can be obtained at:
+
+  * LICENSE:
+    * license/LICENSE.hpack.txt (Apache License 2.0)
+  * HOMEPAGE:
+    * https://github.com/twitter/hpack
+
+This product contains a modified portion of 'Apache Commons Lang', a Java library
+provides utilities for the java.lang API, which can be obtained at:
+
+  * LICENSE:
+    * license/LICENSE.commons-lang.txt (Apache License 2.0)
+  * HOMEPAGE:
+    * https://commons.apache.org/proper/commons-lang/
+
+The binary distribution of this product bundles binaries of
+Commons Codec 1.4,
+which has the following notices:
+ * src/test/org/apache/commons/codec/language/DoubleMetaphoneTest.javacontains test data from http://aspell.net/test/orig/batch0.tab.Copyright (C) 2002 Kevin Atkinson (kevina@gnu.org)
+  ===============================================================================
+  The content of package org.apache.commons.codec.language.bm has been translated
+  from the original php source code available at http://stevemorse.org/phoneticinfo.htm
+  with permission from the original authors.
+  Original source copyright:Copyright (c) 2008 Alexander Beider & Stephen P. Morse.
+
+The binary distribution of this product bundles binaries of
+Commons Lang 2.6,
+which has the following notices:
+ * This product includes software from the Spring Framework,under the Apache License 2.0 (see: StringUtils.containsWhitespace())
+
+The binary distribution of this product bundles binaries of
+Apache Log4j 1.2.17,
+which has the following notices:
+ * ResolverUtil.java
+    Copyright 2005-2006 Tim Fennell
+  Dumbster SMTP test server
+    Copyright 2004 Jason Paul Kitchen
+  TypeUtil.java
+    Copyright 2002-2012 Ramnivas Laddad, Juergen Hoeller, Chris Beams

http://git-wip-us.apache.org/repos/asf/hadoop/blob/098ae11e/hadoop-build-tools/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-build-tools/pom.xml b/hadoop-build-tools/pom.xml
index d08cb89..e84c94f 100644
--- a/hadoop-build-tools/pom.xml
+++ b/hadoop-build-tools/pom.xml
@@ -28,4 +28,45 @@
   <properties>
     <failIfNoTests>false</failIfNoTests>
   </properties>
+  <build>
+    <plugins>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-site-plugin</artifactId>
+        <configuration>
+          <skip>true</skip>
+        </configuration>
+      </plugin>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-remote-resources-plugin</artifactId>
+        <executions>
+          <execution>
+            <goals>
+              <goal>bundle</goal>
+            </goals>
+          </execution>
+        </executions>
+        <configuration>
+          <includes>
+            <include>META-INF/LICENSE.txt</include>
+            <include>META-INF/NOTICE.txt</include>
+          </includes>
+        </configuration>
+      </plugin>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-antrun-plugin</artifactId>
+        <executions>
+          <execution>
+            <id>dummy</id>
+            <phase>validate</phase>
+            <goals>
+              <goal>run</goal>
+            </goals>
+          </execution>
+        </executions>
+      </plugin>
+    </plugins>
+  </build>
 </project>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/098ae11e/hadoop-project-dist/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-project-dist/pom.xml b/hadoop-project-dist/pom.xml
index 2cccbbe..ba4c0a8 100644
--- a/hadoop-project-dist/pom.xml
+++ b/hadoop-project-dist/pom.xml
@@ -80,6 +80,8 @@
               <includes>
                 <include>**/*.class</include>
                 <include>webapps/**</include>
+                <include>META-INF/LICENSE.txt</include>
+                <include>META-INF/NOTICE.txt</include>
               </includes>
             </configuration>
           </execution>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/098ae11e/hadoop-project/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-project/pom.xml b/hadoop-project/pom.xml
index 35166b1..7558b04 100644
--- a/hadoop-project/pom.xml
+++ b/hadoop-project/pom.xml
@@ -111,6 +111,7 @@
     <maven-source-plugin.version>2.3</maven-source-plugin.version>
     <maven-project-info-reports-plugin.version>2.7</maven-project-info-reports-plugin.version>
     <maven-pdf-plugin.version>1.2</maven-pdf-plugin.version>
+    <maven-remote-resources-plugin.version>1.5</maven-remote-resources-plugin.version>
     <build-helper-maven-plugin.version>1.9</build-helper-maven-plugin.version>
     <exec-maven-plugin.version>1.3.1</exec-maven-plugin.version>
     <make-maven-plugin.version>1.0-beta-1</make-maven-plugin.version>
@@ -1134,7 +1135,23 @@
         <groupId>org.codehaus.mojo</groupId>
         <artifactId>findbugs-maven-plugin</artifactId>
       </plugin>
-      
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-remote-resources-plugin</artifactId>
+        <version>${maven-remote-resources-plugin.version}</version>
+        <configuration>
+          <resourceBundles>
+            <resourceBundle>org.apache.hadoop:hadoop-build-tools:${project.version}</resourceBundle>
+          </resourceBundles>
+        </configuration>
+        <executions>
+          <execution>
+            <goals>
+              <goal>process</goal>
+            </goals>
+          </execution>
+        </executions>
+      </plugin>
        <plugin>
          <groupId>org.apache.maven.plugins</groupId>
         <artifactId>maven-antrun-plugin</artifactId>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/098ae11e/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 5bcf993..d86b45c 100644
--- a/pom.xml
+++ b/pom.xml
@@ -413,6 +413,32 @@ xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xs
         <artifactId>dependency-check-maven</artifactId>
         <version>${dependency-check-maven.version}</version>
       </plugin>
+      <plugin>
+        <!-- Copy license and notice files into hadoop-resource-bundle's
+             metadata, which will then be bundled into jars.
+        -->
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-antrun-plugin</artifactId>
+        <executions>
+          <execution>
+            <id>copy-files</id>
+            <phase>process-resources</phase>
+            <goals>
+              <goal>run</goal>
+            </goals>
+            <configuration>
+              <target>
+                <copy todir="${project.build.directory}/../hadoop-build-tools/src/main/resources/META-INF/">
+                  <fileset dir="${project.build.directory}/../">
+                    <include name="LICENSE.txt"/>
+                    <include name="NOTICE.txt"/>
+                  </fileset>
+                </copy>
+              </target>
+            </configuration>
+          </execution>
+        </executions>
+      </plugin>
     </plugins>
   </build>
 


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


[14/39] hadoop git commit: HADOOP-12291. Add support for nested groups in LdapGroupsMapping. Contributed by Esther Kundin.

Posted by ae...@apache.org.
HADOOP-12291. Add support for nested groups in LdapGroupsMapping. Contributed by Esther Kundin.


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

Branch: refs/heads/HDFS-1312
Commit: 6f0aa75121224589fe1e20630c597f851ef3bed2
Parents: 25064fb
Author: Jitendra Pandey <ji...@apache.org>
Authored: Wed Jun 15 11:41:49 2016 -0700
Committer: Jitendra Pandey <ji...@apache.org>
Committed: Wed Jun 15 11:41:49 2016 -0700

----------------------------------------------------------------------
 .../hadoop/security/LdapGroupsMapping.java      | 114 ++++++++++++++++---
 .../src/main/resources/core-default.xml         |  13 +++
 .../hadoop/security/TestLdapGroupsMapping.java  |  62 ++++++++--
 .../security/TestLdapGroupsMappingBase.java     |  33 +++++-
 .../TestLdapGroupsMappingWithPosixGroup.java    |   2 +-
 5 files changed, 198 insertions(+), 26 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/6f0aa751/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/LdapGroupsMapping.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/LdapGroupsMapping.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/LdapGroupsMapping.java
index da87369..5a0b1d9 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/LdapGroupsMapping.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/LdapGroupsMapping.java
@@ -25,6 +25,9 @@ import java.util.ArrayList;
 import java.util.Collections;
 import java.util.Hashtable;
 import java.util.List;
+import java.util.HashSet;
+import java.util.Collection;
+import java.util.Set;
 
 import javax.naming.Context;
 import javax.naming.NamingEnumeration;
@@ -66,9 +69,11 @@ import org.apache.hadoop.conf.Configuration;
  * is used for searching users or groups which returns more results than are
  * allowed by the server, an exception will be thrown.
  * 
- * The implementation also does not attempt to resolve group hierarchies. In
- * order to be considered a member of a group, the user must be an explicit
- * member in LDAP.
+ * The implementation attempts to resolve group hierarchies,
+ * to a configurable limit.
+ * If the limit is 0, in order to be considered a member of a group,
+ * the user must be an explicit member in LDAP.  Otherwise, it will traverse the
+ * group hierarchy n levels up.
  */
 @InterfaceAudience.LimitedPrivate({"HDFS", "MapReduce"})
 @InterfaceStability.Evolving
@@ -157,6 +162,13 @@ public class LdapGroupsMapping
   public static final String GROUP_NAME_ATTR_DEFAULT = "cn";
 
   /*
+   * How many levels to traverse when checking for groups in the org hierarchy
+   */
+  public static final String GROUP_HIERARCHY_LEVELS_KEY =
+        LDAP_CONFIG_PREFIX + ".search.group.hierarchy.levels";
+  public static final int GROUP_HIERARCHY_LEVELS_DEFAULT = 0;
+
+  /*
    * LDAP attribute names to use when doing posix-like lookups
    */
   public static final String POSIX_UID_ATTR_KEY = LDAP_CONFIG_PREFIX + ".posix.attr.uid.name";
@@ -208,6 +220,7 @@ public class LdapGroupsMapping
   private String memberOfAttr;
   private String groupMemberAttr;
   private String groupNameAttr;
+  private int    groupHierarchyLevels;
   private String posixUidAttr;
   private String posixGidAttr;
   private boolean isPosix;
@@ -234,7 +247,7 @@ public class LdapGroupsMapping
      */
     for(int retry = 0; retry < RECONNECT_RETRY_COUNT; retry++) {
       try {
-        return doGetGroups(user);
+        return doGetGroups(user, groupHierarchyLevels);
       } catch (NamingException e) {
         LOG.warn("Failed to get groups for user " + user + " (retry=" + retry
             + ") by " + e);
@@ -324,9 +337,11 @@ public class LdapGroupsMapping
    * @return a list of strings representing group names of the user.
    * @throws NamingException if unable to find group names
    */
-  private List<String> lookupGroup(SearchResult result, DirContext c)
+  private List<String> lookupGroup(SearchResult result, DirContext c,
+      int goUpHierarchy)
       throws NamingException {
     List<String> groups = new ArrayList<String>();
+    Set<String> groupDNs = new HashSet<String>();
 
     NamingEnumeration<SearchResult> groupResults = null;
     // perform the second LDAP query
@@ -345,12 +360,14 @@ public class LdapGroupsMapping
     if (groupResults != null) {
       while (groupResults.hasMoreElements()) {
         SearchResult groupResult = groupResults.nextElement();
-        Attribute groupName = groupResult.getAttributes().get(groupNameAttr);
-        if (groupName == null) {
-          throw new NamingException("The group object does not have " +
-              "attribute '" + groupNameAttr + "'.");
-        }
-        groups.add(groupName.get().toString());
+        getGroupNames(groupResult, groups, groupDNs, goUpHierarchy > 0);
+      }
+      if (goUpHierarchy > 0 && !isPosix) {
+        // convert groups to a set to ensure uniqueness
+        Set<String> groupset = new HashSet<String>(groups);
+        goUpGroupHierarchy(groupDNs, goUpHierarchy, groupset);
+        // convert set back to list for compatibility
+        groups = new ArrayList<String>(groupset);
       }
     }
     return groups;
@@ -369,7 +386,8 @@ public class LdapGroupsMapping
    * return an empty string array.
    * @throws NamingException if unable to get group names
    */
-  List<String> doGetGroups(String user) throws NamingException {
+  List<String> doGetGroups(String user, int goUpHierarchy)
+      throws NamingException {
     DirContext c = getDirContext();
 
     // Search for the user. We'll only ever need to look at the first result
@@ -378,7 +396,7 @@ public class LdapGroupsMapping
     // return empty list if the user can not be found.
     if (!results.hasMoreElements()) {
       if (LOG.isDebugEnabled()) {
-        LOG.debug("doGetGroups(" + user + ") return no groups because the " +
+        LOG.debug("doGetGroups(" + user + ") returned no groups because the " +
             "user is not found.");
       }
       return new ArrayList<String>();
@@ -411,15 +429,76 @@ public class LdapGroupsMapping
                 "the second LDAP query using the user's DN.", e);
       }
     }
-    if (groups == null || groups.isEmpty()) {
-      groups = lookupGroup(result, c);
+    if (groups == null || groups.isEmpty() || goUpHierarchy > 0) {
+      groups = lookupGroup(result, c, goUpHierarchy);
     }
     if (LOG.isDebugEnabled()) {
-      LOG.debug("doGetGroups(" + user + ") return " + groups);
+      LOG.debug("doGetGroups(" + user + ") returned " + groups);
     }
     return groups;
   }
 
+  /* Helper function to get group name from search results.
+  */
+  void getGroupNames(SearchResult groupResult, Collection<String> groups,
+                     Collection<String> groupDNs, boolean doGetDNs)
+                     throws NamingException  {
+    Attribute groupName = groupResult.getAttributes().get(groupNameAttr);
+    if (groupName == null) {
+      throw new NamingException("The group object does not have " +
+        "attribute '" + groupNameAttr + "'.");
+    }
+    groups.add(groupName.get().toString());
+    if (doGetDNs) {
+      groupDNs.add(groupResult.getNameInNamespace());
+    }
+  }
+
+  /* Implementation for walking up the ldap hierarchy
+   * This function will iteratively find the super-group memebership of
+   *    groups listed in groupDNs and add them to
+   * the groups set.  It will walk up the hierarchy goUpHierarchy levels.
+   * Note: This is an expensive operation and settings higher than 1
+   *    are NOT recommended as they will impact both the speed and
+   *    memory usage of all operations.
+   * The maximum time for this function will be bounded by the ldap query
+   * timeout and the number of ldap queries that it will make, which is
+   * max(Recur Depth in LDAP, goUpHierarcy) * DIRECTORY_SEARCH_TIMEOUT
+   *
+   * @param ctx - The context for contacting the ldap server
+   * @param groupDNs - the distinguished name of the groups whose parents we
+   *    want to look up
+   * @param goUpHierarchy - the number of levels to go up,
+   * @param groups - Output variable to store all groups that will be added
+  */
+  void goUpGroupHierarchy(Set<String> groupDNs,
+                          int goUpHierarchy,
+                          Set<String> groups)
+      throws NamingException {
+    if (goUpHierarchy <= 0 || groups.isEmpty()) {
+      return;
+    }
+    DirContext context = getDirContext();
+    Set<String> nextLevelGroups = new HashSet<String>();
+    StringBuilder filter = new StringBuilder();
+    filter.append("(&").append(groupSearchFilter).append("(|");
+    for (String dn : groupDNs) {
+      filter.append("(").append(groupMemberAttr).append("=")
+        .append(dn).append(")");
+    }
+    filter.append("))");
+    LOG.debug("Ldap group query string: " + filter.toString());
+    NamingEnumeration<SearchResult> groupResults =
+        context.search(baseDN,
+           filter.toString(),
+           SEARCH_CONTROLS);
+    while (groupResults.hasMoreElements()) {
+      SearchResult groupResult = groupResults.nextElement();
+      getGroupNames(groupResult, groups, nextLevelGroups, true);
+    }
+    goUpGroupHierarchy(nextLevelGroups, goUpHierarchy - 1, groups);
+  }
+
   DirContext getDirContext() throws NamingException {
     if (ctx == null) {
       // Set up the initial environment for LDAP connectivity
@@ -446,7 +525,6 @@ public class LdapGroupsMapping
 
       ctx = new InitialDirContext(env);
     }
-
     return ctx;
   }
   
@@ -513,6 +591,8 @@ public class LdapGroupsMapping
         conf.get(GROUP_MEMBERSHIP_ATTR_KEY, GROUP_MEMBERSHIP_ATTR_DEFAULT);
     groupNameAttr =
         conf.get(GROUP_NAME_ATTR_KEY, GROUP_NAME_ATTR_DEFAULT);
+    groupHierarchyLevels =
+        conf.getInt(GROUP_HIERARCHY_LEVELS_KEY, GROUP_HIERARCHY_LEVELS_DEFAULT);
     posixUidAttr =
         conf.get(POSIX_UID_ATTR_KEY, POSIX_UID_ATTR_DEFAULT);
     posixGidAttr =

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6f0aa751/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml b/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml
index f1d77dd..089a2c1 100644
--- a/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml
+++ b/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml
@@ -316,6 +316,19 @@
 </property>
 
 <property>
+  <name>hadoop.security.group.mapping.ldap.search.group.hierarchy.levels</name>
+  <value>0</value>
+  <description>
+    The number of levels to go up the group hierarchy when determining
+    which groups a user is part of. 0 Will represent checking just the
+    group that the user belongs to.  Each additional level will raise the
+    time it takes to exectue a query by at most
+    hadoop.security.group.mapping.ldap.directory.search.timeout.
+    The default will usually be appropriate for all LDAP systems.
+  </description>
+</property>
+
+<property>
   <name>hadoop.security.group.mapping.ldap.posix.attr.uid.name</name>
   <value>uidNumber</value>
   <description>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6f0aa751/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestLdapGroupsMapping.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestLdapGroupsMapping.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestLdapGroupsMapping.java
index 9f9f994..131b4e6 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestLdapGroupsMapping.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestLdapGroupsMapping.java
@@ -39,6 +39,7 @@ import java.net.Socket;
 import java.util.Arrays;
 import java.util.List;
 import java.util.concurrent.CountDownLatch;
+import java.util.HashSet;
 
 import javax.naming.CommunicationException;
 import javax.naming.NamingException;
@@ -91,8 +92,23 @@ public class TestLdapGroupsMapping extends TestLdapGroupsMappingBase {
     when(getContext().search(anyString(), anyString(), any(Object[].class),
         any(SearchControls.class)))
         .thenReturn(getUserNames(), getGroupNames());
-    
-    doTestGetGroups(Arrays.asList(testGroups), 2);
+    doTestGetGroups(Arrays.asList(getTestGroups()), 2);
+  }
+
+  @Test
+  public void testGetGroupsWithHierarchy() throws IOException, NamingException {
+    // The search functionality of the mock context is reused, so we will
+    // return the user NamingEnumeration first, and then the group
+    // The parent search is run once for each level, and is a different search
+    // The parent group is returned once for each group, yet the final list
+    // should be unique
+    when(getContext().search(anyString(), anyString(), any(Object[].class),
+        any(SearchControls.class)))
+        .thenReturn(getUserNames(), getGroupNames());
+    when(getContext().search(anyString(), anyString(),
+        any(SearchControls.class)))
+        .thenReturn(getParentGroupNames());
+    doTestGetGroupsWithParent(Arrays.asList(getTestParentGroups()), 2, 1);
   }
 
   @Test
@@ -104,8 +120,10 @@ public class TestLdapGroupsMapping extends TestLdapGroupsMappingBase {
         .thenThrow(new CommunicationException("Connection is closed"))
         .thenReturn(getUserNames(), getGroupNames());
     
-    // Although connection is down but after reconnected it still should retrieve the result groups
-    doTestGetGroups(Arrays.asList(testGroups), 1 + 2); // 1 is the first failure call 
+    // Although connection is down but after reconnected
+    // it still should retrieve the result groups
+    // 1 is the first failure call
+    doTestGetGroups(Arrays.asList(getTestGroups()), 1 + 2);
   }
 
   @Test
@@ -139,7 +157,37 @@ public class TestLdapGroupsMapping extends TestLdapGroupsMappingBase {
                                          any(Object[].class),
                                          any(SearchControls.class));
   }
-  
+
+  private void doTestGetGroupsWithParent(List<String> expectedGroups,
+      int searchTimesGroup, int searchTimesParentGroup)
+          throws IOException, NamingException {
+    Configuration conf = new Configuration();
+    // Set this, so we don't throw an exception
+    conf.set(LdapGroupsMapping.LDAP_URL_KEY, "ldap://test");
+    // Set the config to get parents 1 level up
+    conf.setInt(LdapGroupsMapping.GROUP_HIERARCHY_LEVELS_KEY, 1);
+
+    LdapGroupsMapping groupsMapping = getGroupsMapping();
+    groupsMapping.setConf(conf);
+    // Username is arbitrary, since the spy is mocked to respond the same,
+    // regardless of input
+    List<String> groups = groupsMapping.getGroups("some_user");
+
+    // compare lists, ignoring the order
+    Assert.assertEquals(new HashSet<String>(expectedGroups),
+        new HashSet<String>(groups));
+
+    // We should have searched for a user, and group
+    verify(getContext(), times(searchTimesGroup)).search(anyString(),
+                                         anyString(),
+                                         any(Object[].class),
+                                         any(SearchControls.class));
+    // One groups search for the parent group should have been done
+    verify(getContext(), times(searchTimesParentGroup)).search(anyString(),
+                                         anyString(),
+                                         any(SearchControls.class));
+  }
+
   @Test
   public void testExtractPassword() throws IOException {
     File testDir = GenericTestUtils.getTestDir();
@@ -246,7 +294,7 @@ public class TestLdapGroupsMapping extends TestLdapGroupsMappingBase {
       mapping.setConf(conf);
 
       try {
-        mapping.doGetGroups("hadoop");
+        mapping.doGetGroups("hadoop", 1);
         fail("The LDAP query should have timed out!");
       } catch (NamingException ne) {
         LOG.debug("Got the exception while LDAP querying: ", ne);
@@ -302,7 +350,7 @@ public class TestLdapGroupsMapping extends TestLdapGroupsMappingBase {
       mapping.setConf(conf);
 
       try {
-        mapping.doGetGroups("hadoop");
+        mapping.doGetGroups("hadoop", 1);
         fail("The LDAP query should have timed out!");
       } catch (NamingException ne) {
         LOG.debug("Got the exception while LDAP querying: ", ne);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6f0aa751/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestLdapGroupsMappingBase.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestLdapGroupsMappingBase.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestLdapGroupsMappingBase.java
index 75e3bf1..51d3673 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestLdapGroupsMappingBase.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestLdapGroupsMappingBase.java
@@ -46,13 +46,17 @@ public class TestLdapGroupsMappingBase {
   @Mock
   private NamingEnumeration<SearchResult> groupNames;
   @Mock
+  private NamingEnumeration<SearchResult> parentGroupNames;
+  @Mock
   private SearchResult userSearchResult;
   @Mock
   private Attributes attributes;
   @Spy
   private LdapGroupsMapping groupsMapping = new LdapGroupsMapping();
 
-  protected String[] testGroups = new String[] {"group1", "group2"};
+  private String[] testGroups = new String[] {"group1", "group2"};
+  private String[] testParentGroups =
+      new String[] {"group1", "group2", "group1_1"};
 
   @Before
   public void setupMocksBase() throws NamingException {
@@ -93,6 +97,24 @@ public class TestLdapGroupsMappingBase {
         thenReturn(getUserSearchResult());
 
     when(getUserSearchResult().getAttributes()).thenReturn(getAttributes());
+    // Define results for groups 1 level up
+    SearchResult parentGroupResult = mock(SearchResult.class);
+
+    // only one parent group
+    when(parentGroupNames.hasMoreElements()).thenReturn(true, false);
+    when(parentGroupNames.nextElement()).
+        thenReturn(parentGroupResult);
+
+    // Define the attribute for the parent group
+    Attribute parentGroup1Attr = new BasicAttribute("cn");
+    parentGroup1Attr.add(testParentGroups[2]);
+    Attributes parentGroup1Attrs = new BasicAttributes();
+    parentGroup1Attrs.put(parentGroup1Attr);
+
+    // attach the attributes to the result
+    when(parentGroupResult.getAttributes()).thenReturn(parentGroup1Attrs);
+    when(parentGroupResult.getNameInNamespace()).
+        thenReturn("CN=some_group,DC=test,DC=com");
   }
 
   protected DirContext getContext() {
@@ -117,4 +139,13 @@ public class TestLdapGroupsMappingBase {
   protected LdapGroupsMapping getGroupsMapping() {
     return groupsMapping;
   }
+  protected String[] getTestGroups() {
+    return testGroups;
+  }
+  protected NamingEnumeration getParentGroupNames() {
+    return parentGroupNames;
+  }
+  protected String[] getTestParentGroups() {
+    return testParentGroups;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6f0aa751/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestLdapGroupsMappingWithPosixGroup.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestLdapGroupsMappingWithPosixGroup.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestLdapGroupsMappingWithPosixGroup.java
index 332eed4..17d28a5 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestLdapGroupsMappingWithPosixGroup.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestLdapGroupsMappingWithPosixGroup.java
@@ -69,7 +69,7 @@ public class TestLdapGroupsMappingWithPosixGroup
         any(Object[].class), any(SearchControls.class)))
         .thenReturn(getUserNames(), getGroupNames());
 
-    doTestGetGroups(Arrays.asList(testGroups), 2);
+    doTestGetGroups(Arrays.asList(getTestGroups()), 2);
   }
 
   private void doTestGetGroups(List<String> expectedGroups, int searchTimes)


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


[32/39] hadoop git commit: YARN-5251. Yarn CLI to obtain App logs for last 'n' bytes fails. Contributed by Xuan Gong.

Posted by ae...@apache.org.
YARN-5251. Yarn CLI to obtain App logs for last 'n' bytes fails. Contributed by Xuan Gong.


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

Branch: refs/heads/HDFS-1312
Commit: c35fa4a0e57b99415e284c7c2f3b1a0bebc610c2
Parents: 09e82ac
Author: Junping Du <ju...@apache.org>
Authored: Fri Jun 17 08:24:24 2016 -0700
Committer: Junping Du <ju...@apache.org>
Committed: Fri Jun 17 08:24:24 2016 -0700

----------------------------------------------------------------------
 .../hadoop/yarn/client/cli/TestLogsCLI.java     |  2 +-
 .../logaggregation/AggregatedLogFormat.java     | 26 +++++++++++---------
 .../logaggregation/TestAggregatedLogFormat.java |  2 +-
 .../webapp/AHSWebServices.java                  | 11 +++++----
 .../nodemanager/webapp/NMWebServices.java       |  9 +++----
 .../TestLogAggregationService.java              |  2 +-
 6 files changed, 27 insertions(+), 25 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/c35fa4a0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestLogsCLI.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestLogsCLI.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestLogsCLI.java
index df1c6b7..6683fc7 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestLogsCLI.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestLogsCLI.java
@@ -400,7 +400,7 @@ public class TestLogsCLI {
 
     String logMessage = "Hello container_0_0001_01_000003 in stdout!";
     int fileContentSize = logMessage.getBytes().length;
-    int tailContentSize = "End of LogType:syslog\n\n".getBytes().length;
+    int tailContentSize = "\nEnd of LogType:syslog\n\n".getBytes().length;
 
     // specify how many bytes we should get from logs
     // specify a position number, it would get the first n bytes from

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c35fa4a0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/AggregatedLogFormat.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/AggregatedLogFormat.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/AggregatedLogFormat.java
index 2fb18ed..7508dd5 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/AggregatedLogFormat.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/AggregatedLogFormat.java
@@ -787,20 +787,19 @@ public class AggregatedLogFormat {
 
       long toSkip = 0;
       long totalBytesToRead = fileLength;
+      long skipAfterRead = 0;
       if (bytes < 0) {
         long absBytes = Math.abs(bytes);
         if (absBytes < fileLength) {
           toSkip = fileLength - absBytes;
           totalBytesToRead = absBytes;
         }
-        long skippedBytes = valueStream.skip(toSkip);
-        if (skippedBytes != toSkip) {
-          throw new IOException("The bytes were skipped are "
-              + "different from the caller requested");
-        }
+        org.apache.hadoop.io.IOUtils.skipFully(
+            valueStream, toSkip);
       } else {
         if (bytes < fileLength) {
           totalBytesToRead = bytes;
+          skipAfterRead = fileLength - bytes;
         }
       }
 
@@ -818,7 +817,9 @@ public class AggregatedLogFormat {
                   pendingRead > buf.length ? buf.length : (int) pendingRead;
         len = valueStream.read(buf, 0, toRead);
       }
-      out.println("End of LogType:" + fileType);
+      org.apache.hadoop.io.IOUtils.skipFully(
+          valueStream, skipAfterRead);
+      out.println("\nEnd of LogType:" + fileType);
       out.println("");
     }
 
@@ -913,20 +914,19 @@ public class AggregatedLogFormat {
 
         long toSkip = 0;
         long totalBytesToRead = fileLength;
+        long skipAfterRead = 0;
         if (bytes < 0) {
           long absBytes = Math.abs(bytes);
           if (absBytes < fileLength) {
             toSkip = fileLength - absBytes;
             totalBytesToRead = absBytes;
           }
-          long skippedBytes = valueStream.skip(toSkip);
-          if (skippedBytes != toSkip) {
-            throw new IOException("The bytes were skipped are "
-                + "different from the caller requested");
-          }
+          org.apache.hadoop.io.IOUtils.skipFully(
+              valueStream, toSkip);
         } else {
           if (bytes < fileLength) {
             totalBytesToRead = bytes;
+            skipAfterRead = fileLength - bytes;
           }
         }
 
@@ -942,7 +942,9 @@ public class AggregatedLogFormat {
           toRead = pendingRead > buf.length ? buf.length : (int) pendingRead;
           len = valueStream.read(buf, 0, toRead);
         }
-        out.println("End of LogType:" + fileType);
+        org.apache.hadoop.io.IOUtils.skipFully(
+            valueStream, skipAfterRead);
+        out.println("\nEnd of LogType:" + fileType);
         out.println("");
         return 0;
       } else {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c35fa4a0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/logaggregation/TestAggregatedLogFormat.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/logaggregation/TestAggregatedLogFormat.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/logaggregation/TestAggregatedLogFormat.java
index 21e1655..45dd8ab 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/logaggregation/TestAggregatedLogFormat.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/logaggregation/TestAggregatedLogFormat.java
@@ -258,7 +258,7 @@ public class TestAggregatedLogFormat {
               .currentTimeMillis())).length() : 0)
             + ("\nLogLength:" + numChars).length()
             + "\nLog Contents:\n".length() + numChars + "\n".length()
-            + "End of LogType:stdout\n".length();
+            + "\nEnd of LogType:stdout\n".length();
     Assert.assertTrue("LogType not matched", s.contains("LogType:stdout"));
     Assert.assertTrue("log file:stderr should not be aggregated.", !s.contains("LogType:stderr"));
     Assert.assertTrue("log file:logs should not be aggregated.", !s.contains("LogType:logs"));

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c35fa4a0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/AHSWebServices.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/AHSWebServices.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/AHSWebServices.java
index 692b172..b56fae9 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/AHSWebServices.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/AHSWebServices.java
@@ -404,20 +404,19 @@ public class AHSWebServices extends WebServices {
 
                     long toSkip = 0;
                     long totalBytesToRead = fileLength;
+                    long skipAfterRead = 0;
                     if (bytes < 0) {
                       long absBytes = Math.abs(bytes);
                       if (absBytes < fileLength) {
                         toSkip = fileLength - absBytes;
                         totalBytesToRead = absBytes;
                       }
-                      long skippedBytes = valueStream.skip(toSkip);
-                      if (skippedBytes != toSkip) {
-                        throw new IOException("The bytes were skipped are "
-                            + "different from the caller requested");
-                      }
+                      org.apache.hadoop.io.IOUtils.skipFully(
+                          valueStream, toSkip);
                     } else {
                       if (bytes < fileLength) {
                         totalBytesToRead = bytes;
+                        skipAfterRead = fileLength - bytes;
                       }
                     }
 
@@ -435,6 +434,8 @@ public class AHSWebServices extends WebServices {
                           : (int) pendingRead;
                       len = valueStream.read(buf, 0, toRead);
                     }
+                    org.apache.hadoop.io.IOUtils.skipFully(
+                        valueStream, skipAfterRead);
                     sb = new StringBuilder();
                     sb.append("\nEnd of LogType:" + fileType + "\n");
                     b = sb.toString().getBytes(Charset.forName("UTF-8"));

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c35fa4a0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/NMWebServices.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/webapp/NMWebServices.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/NMWebServices.java
index 3a30392..4e38e90 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/NMWebServices.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/NMWebServices.java
@@ -264,20 +264,18 @@ public class NMWebServices {
             byte[] buf = new byte[bufferSize];
             long toSkip = 0;
             long totalBytesToRead = fileLength;
+            long skipAfterRead = 0;
             if (bytes < 0) {
               long absBytes = Math.abs(bytes);
               if (absBytes < fileLength) {
                 toSkip = fileLength - absBytes;
                 totalBytesToRead = absBytes;
               }
-              long skippedBytes = fis.skip(toSkip);
-              if (skippedBytes != toSkip) {
-                throw new IOException("The bytes were skipped are different "
-                    + "from the caller requested");
-              }
+              org.apache.hadoop.io.IOUtils.skipFully(fis, toSkip);
             } else {
               if (bytes < fileLength) {
                 totalBytesToRead = bytes;
+                skipAfterRead = fileLength - bytes;
               }
             }
 
@@ -295,6 +293,7 @@ public class NMWebServices {
                   : (int) pendingRead;
               len = fis.read(buf, 0, toRead);
             }
+            org.apache.hadoop.io.IOUtils.skipFully(fis, skipAfterRead);
             os.flush();
           } finally {
             IOUtils.closeQuietly(fis);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c35fa4a0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/logaggregation/TestLogAggregationService.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/logaggregation/TestLogAggregationService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/logaggregation/TestLogAggregationService.java
index c98e366..1edb841 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/logaggregation/TestLogAggregationService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/logaggregation/TestLogAggregationService.java
@@ -948,7 +948,7 @@ public class TestLogAggregationService extends BaseContainerManagerTest {
         Assert.assertEquals(numOfLogsPerContainer, thisContainerMap.size());
         for (String fileType : logFiles) {
           String expectedValue =
-              containerStr + " Hello " + fileType + "!End of LogType:"
+              containerStr + " Hello " + fileType + "!\nEnd of LogType:"
                   + fileType;
           LOG.info("Expected log-content : " + new String(expectedValue));
           String foundValue = thisContainerMap.remove(fileType);


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


[35/39] hadoop git commit: YARN-5082. Limit ContainerId increase in fair scheduler if the num of node app reserved reached the limit. Addendum to fix javac warning. (Arun Suresh via kasha)

Posted by ae...@apache.org.
YARN-5082. Limit ContainerId increase in fair scheduler if the num of node app reserved reached the limit. Addendum to fix javac warning. (Arun Suresh via kasha)


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

Branch: refs/heads/HDFS-1312
Commit: fbbe0bb627c1b5f333eb5d5390e709cc43517420
Parents: 0761379
Author: Karthik Kambatla <ka...@apache.org>
Authored: Fri Jun 17 22:12:50 2016 -0700
Committer: Karthik Kambatla <ka...@apache.org>
Committed: Fri Jun 17 22:12:50 2016 -0700

----------------------------------------------------------------------
 .../server/resourcemanager/scheduler/fair/TestFairScheduler.java | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/fbbe0bb6/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFairScheduler.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFairScheduler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFairScheduler.java
index ec77a9b..9f22e50 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFairScheduler.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFairScheduler.java
@@ -4512,7 +4512,7 @@ public class TestFairScheduler extends FairSchedulerTestBase {
     scheduler.handle(new NodeUpdateSchedulerEvent(node2));
 
     assertEquals(4096, scheduler.getQueueManager().getQueue("queue1").
-        getResourceUsage().getMemory());
+        getResourceUsage().getMemorySize());
 
     //container will be reserved at node1
     RMContainer reservedContainer1 =
@@ -4532,7 +4532,7 @@ public class TestFairScheduler extends FairSchedulerTestBase {
         app1, RMAppAttemptState.KILLED, false));
 
     assertEquals(0, scheduler.getQueueManager().getQueue("queue1").
-        getResourceUsage().getMemory());
+        getResourceUsage().getMemorySize());
 
     // container will be allocated at node2
     scheduler.handle(new NodeUpdateSchedulerEvent(node2));


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


[19/39] hadoop git commit: HDFS-10256. Use GenericTestUtils.getTestDir method in tests for temporary directories (Contributed by Vinayakumar B)

Posted by ae...@apache.org.
HDFS-10256. Use GenericTestUtils.getTestDir method in tests for temporary directories (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/cc6c2651
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/cc6c2651
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/cc6c2651

Branch: refs/heads/HDFS-1312
Commit: cc6c265171aace1e57653e777a4a73a747221086
Parents: 2ca7344
Author: Vinayakumar B <vi...@apache.org>
Authored: Thu Jun 16 16:47:06 2016 +0530
Committer: Vinayakumar B <vi...@apache.org>
Committed: Thu Jun 16 16:47:06 2016 +0530

----------------------------------------------------------------------
 .../java/org/apache/hadoop/fs/FileUtil.java     | 14 +++++
 .../org/apache/hadoop/fs/TestGetSpaceUsed.java  |  5 +-
 .../hadoop/security/token/TestDtUtilShell.java  |  3 +-
 .../hdfs/shortcircuit/TestShortCircuitShm.java  |  4 +-
 .../TestHttpFSFWithSWebhdfsFileSystem.java      |  5 +-
 .../TestHttpFSFileSystemLocalFileSystem.java    |  5 +-
 .../hdfs/nfs/nfs3/TestNfs3HttpServer.java       |  5 +-
 .../apache/hadoop/cli/TestCryptoAdminCLI.java   |  4 +-
 .../hadoop/fs/TestEnhancedByteBufferAccess.java | 12 ++--
 .../TestSWebHdfsFileContextMainOperations.java  |  6 +-
 .../TestWebHdfsFileContextMainOperations.java   |  7 ++-
 .../org/apache/hadoop/hdfs/MiniDFSCluster.java  | 17 +++--
 .../hadoop/hdfs/TestAppendSnapshotTruncate.java |  4 +-
 .../org/apache/hadoop/hdfs/TestDFSShell.java    |  4 +-
 .../hadoop/hdfs/TestDFSUpgradeFromImage.java    | 18 +++---
 .../org/apache/hadoop/hdfs/TestDFSUtil.java     |  3 +-
 .../hadoop/hdfs/TestDatanodeLayoutUpgrade.java  | 21 ++++---
 ...estDatanodeStartupFixesLegacyStorageIDs.java | 12 ++--
 .../org/apache/hadoop/hdfs/TestFetchImage.java  |  5 +-
 .../java/org/apache/hadoop/hdfs/TestPread.java  |  2 +-
 .../org/apache/hadoop/hdfs/TestSeekBug.java     |  3 +-
 .../sasl/SaslDataTransferTestCase.java          |  5 +-
 .../hdfs/qjournal/TestSecureNNWithQJM.java      |  5 +-
 .../hdfs/server/balancer/TestBalancer.java      |  3 +-
 .../hdfs/server/datanode/TestDataNodeUUID.java  |  3 +-
 .../hdfs/server/datanode/TestDataStorage.java   |  3 +-
 .../server/namenode/TestCreateEditsLog.java     |  6 +-
 .../namenode/TestDecommissioningStatus.java     | 65 ++++++--------------
 .../namenode/TestEditLogFileInputStream.java    |  3 +-
 .../namenode/TestFSImageWithSnapshot.java       |  4 +-
 .../hadoop/hdfs/server/namenode/TestFsck.java   | 11 ++--
 .../server/namenode/TestNameNodeHttpServer.java |  5 +-
 .../server/namenode/TestNameNodeRecovery.java   |  5 +-
 .../TestNameNodeRespectsBindHostKeys.java       |  5 +-
 .../snapshot/TestRenameWithSnapshots.java       |  2 +-
 .../server/namenode/snapshot/TestSnapshot.java  |  4 +-
 .../apache/hadoop/hdfs/tools/TestGetConf.java   | 62 ++++---------------
 .../hadoop/hdfs/util/HostsFileWriter.java       | 32 +++++++---
 .../hdfs/util/TestCombinedHostsFileReader.java  |  5 +-
 .../hadoop/hdfs/web/TestHttpsFileSystem.java    |  5 +-
 .../hadoop/hdfs/web/TestWebHdfsTokens.java      |  9 +--
 .../java/org/apache/hadoop/test/PathUtils.java  |  7 +--
 42 files changed, 189 insertions(+), 219 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/cc6c2651/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileUtil.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileUtil.java
index e2d6ecd..53ced77 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileUtil.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileUtil.java
@@ -99,6 +99,20 @@ public class FileUtil {
   }
 
   /**
+   * Register all files recursively to be deleted on exit.
+   * @param file File/directory to be deleted
+   */
+  public static void fullyDeleteOnExit(final File file) {
+    file.deleteOnExit();
+    if (file.isDirectory()) {
+      File[] files = file.listFiles();
+      for (File child : files) {
+        fullyDeleteOnExit(child);
+      }
+    }
+  }
+
+  /**
    * Delete a directory and all its contents.  If
    * we return false, the directory may be partially-deleted.
    * (1) If dir is symlink to a file, the symlink is deleted. The file pointed

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cc6c2651/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestGetSpaceUsed.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestGetSpaceUsed.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestGetSpaceUsed.java
index f436713..d696dbf 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestGetSpaceUsed.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestGetSpaceUsed.java
@@ -18,6 +18,7 @@
 package org.apache.hadoop.fs;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.test.GenericTestUtils;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
@@ -28,8 +29,8 @@ import java.io.IOException;
 import static org.junit.Assert.*;
 
 public class TestGetSpaceUsed {
-  final static private File DIR = new File(
-      System.getProperty("test.build.data", "/tmp"), "TestGetSpaceUsed");
+  final static private File DIR =
+      GenericTestUtils.getTestDir("TestGetSpaceUsed");
 
   @Before
   public void setUp() {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cc6c2651/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/token/TestDtUtilShell.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/token/TestDtUtilShell.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/token/TestDtUtilShell.java
index 5f8f830..1c25912 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/token/TestDtUtilShell.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/token/TestDtUtilShell.java
@@ -31,6 +31,7 @@ import org.apache.hadoop.io.Text;
 import org.apache.hadoop.security.Credentials;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.security.token.TokenIdentifier;
+import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.security.token.DtFetcher;
 
 import static org.junit.Assert.assertEquals;
@@ -71,7 +72,7 @@ public class TestDtUtilShell {
 
   private final ByteArrayOutputStream outContent = new ByteArrayOutputStream();
   private final Path workDir = new Path(
-             System.getProperty("test.build.data", "/tmp"), "TestDtUtilShell");
+      GenericTestUtils.getTestDir("TestDtUtilShell").getAbsolutePath());
   private final Path tokenFile = new Path(workDir, "testPrintTokenFile");
   private final Path tokenFile2 = new Path(workDir, "testPrintTokenFile2");
   private final Path tokenLegacyFile = new Path(workDir, "testPrintTokenFile3");

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cc6c2651/hadoop-hdfs-project/hadoop-hdfs-client/src/test/java/org/apache/hadoop/hdfs/shortcircuit/TestShortCircuitShm.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/test/java/org/apache/hadoop/hdfs/shortcircuit/TestShortCircuitShm.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/test/java/org/apache/hadoop/hdfs/shortcircuit/TestShortCircuitShm.java
index 9d48444..0c202ed 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/test/java/org/apache/hadoop/hdfs/shortcircuit/TestShortCircuitShm.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/test/java/org/apache/hadoop/hdfs/shortcircuit/TestShortCircuitShm.java
@@ -22,6 +22,7 @@ import org.apache.hadoop.hdfs.ExtendedBlockId;
 import org.apache.hadoop.hdfs.shortcircuit.ShortCircuitShm.ShmId;
 import org.apache.hadoop.hdfs.shortcircuit.ShortCircuitShm.Slot;
 import org.apache.hadoop.io.nativeio.SharedFileDescriptorFactory;
+import org.apache.hadoop.test.GenericTestUtils;
 import org.junit.Assert;
 import org.junit.Assume;
 import org.junit.Before;
@@ -38,8 +39,7 @@ public class TestShortCircuitShm {
   public static final Logger LOG = LoggerFactory.getLogger(
       TestShortCircuitShm.class);
   
-  private static final File TEST_BASE =
-      new File(System.getProperty("test.build.data", "/tmp"));
+  private static final File TEST_BASE = GenericTestUtils.getTestDir();
 
   @Before
   public void before() {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cc6c2651/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/client/TestHttpFSFWithSWebhdfsFileSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/client/TestHttpFSFWithSWebhdfsFileSystem.java b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/client/TestHttpFSFWithSWebhdfsFileSystem.java
index 9c3d700..d8f860a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/client/TestHttpFSFWithSWebhdfsFileSystem.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/client/TestHttpFSFWithSWebhdfsFileSystem.java
@@ -23,6 +23,7 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.hdfs.web.SWebHdfsFileSystem;
 import org.apache.hadoop.security.ssl.KeyStoreTestUtil;
+import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.test.TestJettyHelper;
 import org.junit.AfterClass;
 import org.junit.runner.RunWith;
@@ -37,8 +38,8 @@ import java.util.UUID;
 public class TestHttpFSFWithSWebhdfsFileSystem
   extends TestHttpFSWithHttpFSFileSystem {
   private static String classpathDir;
-  private static final String BASEDIR = System.getProperty("test.build.dir",
-      "target/test-dir") + "/" + UUID.randomUUID();
+  private static final String BASEDIR =
+      GenericTestUtils.getTempPath(UUID.randomUUID().toString());
 
   private static Configuration sslConf;
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cc6c2651/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/client/TestHttpFSFileSystemLocalFileSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/client/TestHttpFSFileSystemLocalFileSystem.java b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/client/TestHttpFSFileSystemLocalFileSystem.java
index e9a80e3..83bcb2e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/client/TestHttpFSFileSystemLocalFileSystem.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/client/TestHttpFSFileSystemLocalFileSystem.java
@@ -25,6 +25,7 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.permission.FsAction;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.test.TestDirHelper;
 import org.junit.Assert;
 import org.junit.runner.RunWith;
@@ -41,9 +42,7 @@ public class TestHttpFSFileSystemLocalFileSystem extends BaseTestHttpFSWith {
 
   static {
     new TestDirHelper();
-    String prefix =
-      System.getProperty("test.build.dir", "target/test-dir") + "/local";
-    File file = new File(prefix);
+    File file = GenericTestUtils.getTestDir("local");
     file.mkdirs();
     PATH_PREFIX = file.getAbsolutePath();
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cc6c2651/hadoop-hdfs-project/hadoop-hdfs-nfs/src/test/java/org/apache/hadoop/hdfs/nfs/nfs3/TestNfs3HttpServer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-nfs/src/test/java/org/apache/hadoop/hdfs/nfs/nfs3/TestNfs3HttpServer.java b/hadoop-hdfs-project/hadoop-hdfs-nfs/src/test/java/org/apache/hadoop/hdfs/nfs/nfs3/TestNfs3HttpServer.java
index 46dbd42..952aae2 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-nfs/src/test/java/org/apache/hadoop/hdfs/nfs/nfs3/TestNfs3HttpServer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-nfs/src/test/java/org/apache/hadoop/hdfs/nfs/nfs3/TestNfs3HttpServer.java
@@ -30,13 +30,14 @@ import org.apache.hadoop.hdfs.nfs.conf.NfsConfigKeys;
 import org.apache.hadoop.hdfs.nfs.conf.NfsConfiguration;
 import org.apache.hadoop.http.HttpConfig;
 import org.apache.hadoop.security.ssl.KeyStoreTestUtil;
+import org.apache.hadoop.test.GenericTestUtils;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.Test;
 
 public class TestNfs3HttpServer {
-  private static final String BASEDIR = System.getProperty("test.build.dir",
-      "target/test-dir") + "/" + TestNfs3HttpServer.class.getSimpleName();
+  private static final String BASEDIR =
+      GenericTestUtils.getTempPath(TestNfs3HttpServer.class.getSimpleName());
   private static NfsConfiguration conf = new NfsConfiguration();
   private static MiniDFSCluster cluster;
   private static String keystoresDir;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cc6c2651/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/cli/TestCryptoAdminCLI.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/cli/TestCryptoAdminCLI.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/cli/TestCryptoAdminCLI.java
index bd5349d..7c6aaeb 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/cli/TestCryptoAdminCLI.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/cli/TestCryptoAdminCLI.java
@@ -44,6 +44,7 @@ import org.apache.hadoop.hdfs.HDFSPolicyProvider;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.tools.CryptoAdmin;
 import org.apache.hadoop.security.authorize.PolicyProvider;
+import org.apache.hadoop.test.GenericTestUtils;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
@@ -63,8 +64,7 @@ public class TestCryptoAdminCLI extends CLITestHelperDFS {
         HDFSPolicyProvider.class, PolicyProvider.class);
     conf.setInt(DFSConfigKeys.DFS_REPLICATION_KEY, 1);
 
-    tmpDir = new File(System.getProperty("test.build.data", "target"),
-        UUID.randomUUID().toString()).getAbsoluteFile();
+    tmpDir = GenericTestUtils.getTestDir(UUID.randomUUID().toString());
     final Path jksPath = new Path(tmpDir.toString(), "test.jks");
     conf.set(DFSConfigKeys.DFS_ENCRYPTION_KEY_PROVIDER_URI,
         JavaKeyStoreProvider.SCHEME_NAME + "://file" + jksPath.toUri());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cc6c2651/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/TestEnhancedByteBufferAccess.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/TestEnhancedByteBufferAccess.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/TestEnhancedByteBufferAccess.java
index 32a34e8..0ccc07a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/TestEnhancedByteBufferAccess.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/TestEnhancedByteBufferAccess.java
@@ -557,27 +557,25 @@ public class TestEnhancedByteBufferAccess {
    */
   @Test
   public void testIndirectFallbackReads() throws Exception {
-    final File TEST_DIR = new File(
-      System.getProperty("test.build.data","build/test/data"));
-    final String TEST_PATH = TEST_DIR + File.separator +
-        "indirectFallbackTestFile";
+    final String testPath = GenericTestUtils
+        .getTestDir("indirectFallbackTestFile").getAbsolutePath();
     final int TEST_FILE_LENGTH = 16385;
     final int RANDOM_SEED = 23453;
     FileOutputStream fos = null;
     FileInputStream fis = null;
     try {
-      fos = new FileOutputStream(TEST_PATH);
+      fos = new FileOutputStream(testPath);
       Random random = new Random(RANDOM_SEED);
       byte original[] = new byte[TEST_FILE_LENGTH];
       random.nextBytes(original);
       fos.write(original);
       fos.close();
       fos = null;
-      fis = new FileInputStream(TEST_PATH);
+      fis = new FileInputStream(testPath);
       testFallbackImpl(fis, original);
     } finally {
       IOUtils.cleanup(LOG, fos, fis);
-      new File(TEST_PATH).delete();
+      new File(testPath).delete();
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cc6c2651/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/TestSWebHdfsFileContextMainOperations.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/TestSWebHdfsFileContextMainOperations.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/TestSWebHdfsFileContextMainOperations.java
index 02110ac..84bd98b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/TestSWebHdfsFileContextMainOperations.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/TestSWebHdfsFileContextMainOperations.java
@@ -24,6 +24,7 @@ import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.ssl.KeyStoreTestUtil;
 import org.apache.hadoop.security.ssl.SSLFactory;
+import org.apache.hadoop.test.GenericTestUtils;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 
@@ -50,9 +51,8 @@ public class TestSWebHdfsFileContextMainOperations
 
   private static final HdfsConfiguration CONF = new HdfsConfiguration();
 
-  private static final String BASEDIR =
-      System.getProperty("test.build.dir", "target/test-dir") + "/"
-          + TestSWebHdfsFileContextMainOperations.class.getSimpleName();
+  private static final String BASEDIR = GenericTestUtils
+      .getTempPath(TestSWebHdfsFileContextMainOperations.class.getSimpleName());
   protected static int numBlocks = 2;
   protected static final byte[] data = getFileData(numBlocks,
       getDefaultBlockSize());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cc6c2651/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/TestWebHdfsFileContextMainOperations.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/TestWebHdfsFileContextMainOperations.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/TestWebHdfsFileContextMainOperations.java
index ec91cd1..72fc6e6 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/TestWebHdfsFileContextMainOperations.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/TestWebHdfsFileContextMainOperations.java
@@ -23,6 +23,7 @@ import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.test.GenericTestUtils;
 import org.junit.AfterClass;
 import org.junit.Before;
 import org.junit.BeforeClass;
@@ -63,7 +64,7 @@ public class TestWebHdfsFileContextMainOperations
 
   @Override
   protected FileContextTestHelper createFileContextHelper() {
-    return new FileContextTestHelper("/tmp/TestWebHdfsFileContextMainOperations");
+    return new FileContextTestHelper();
   }
 
   public URI getWebhdfsUrl() {
@@ -88,8 +89,8 @@ public class TestWebHdfsFileContextMainOperations
   public void setUp() throws Exception {
     URI webhdfsUrlReal = getWebhdfsUrl();
     Path testBuildData = new Path(
-        webhdfsUrlReal + "/build/test/data/" + RandomStringUtils
-            .randomAlphanumeric(10));
+        webhdfsUrlReal + "/" + GenericTestUtils.DEFAULT_TEST_DATA_PATH
+            + RandomStringUtils.randomAlphanumeric(10));
     Path rootPath = new Path(testBuildData, "root-uri");
 
     localFsRootPath = rootPath.makeQualified(webhdfsUrlReal, null);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cc6c2651/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 ae6a133..3b3a88b 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
@@ -103,7 +103,6 @@ import org.apache.hadoop.hdfs.server.datanode.SecureDataNodeStarter.SecureResour
 import org.apache.hadoop.hdfs.server.datanode.SimulatedFSDataset;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsDatasetSpi;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi;
-import org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.FsDatasetUtil;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.FsVolumeImpl;
 import org.apache.hadoop.hdfs.server.namenode.EditLogFileOutputStream;
 import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
@@ -142,7 +141,8 @@ public class MiniDFSCluster {
   private static final String NAMESERVICE_ID_PREFIX = "nameserviceId";
   private static final Log LOG = LogFactory.getLog(MiniDFSCluster.class);
   /** System property to set the data dir: {@value} */
-  public static final String PROP_TEST_BUILD_DATA = "test.build.data";
+  public static final String PROP_TEST_BUILD_DATA =
+      GenericTestUtils.SYSPROP_TEST_DATA_DIR;
   /** Configuration option to set the data dir: {@value} */
   public static final String HDFS_MINIDFS_BASEDIR = "hdfs.minidfs.basedir";
   public static final String  DFS_NAMENODE_SAFEMODE_EXTENSION_TESTING_KEY
@@ -1931,12 +1931,11 @@ public class MiniDFSCluster {
     ShutdownHookManager.get().clearShutdownHooks();
     if (base_dir != null) {
       if (deleteDfsDir) {
-        base_dir.delete();
+        FileUtil.fullyDelete(base_dir);
       } else {
-        base_dir.deleteOnExit();
+        FileUtil.fullyDeleteOnExit(base_dir);
       }
     }
-
   }
   
   /**
@@ -2738,13 +2737,13 @@ public class MiniDFSCluster {
 
   /**
    * Get the base directory for any DFS cluster whose configuration does
-   * not explicitly set it. This is done by retrieving the system property
-   * {@link #PROP_TEST_BUILD_DATA} (defaulting to "build/test/data" ),
-   * and returning that directory with a subdir of /dfs.
+   * not explicitly set it. This is done via
+   * {@link GenericTestUtils#getTestDir()}.
    * @return a directory for use as a miniDFS filesystem.
    */
   public static String getBaseDirectory() {
-    return System.getProperty(PROP_TEST_BUILD_DATA, "build/test/data") + "/dfs/";
+    return GenericTestUtils.getTestDir("dfs").getAbsolutePath()
+        + File.separator;
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cc6c2651/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestAppendSnapshotTruncate.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestAppendSnapshotTruncate.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestAppendSnapshotTruncate.java
index f41f809..0280f8c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestAppendSnapshotTruncate.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestAppendSnapshotTruncate.java
@@ -115,9 +115,7 @@ public class TestAppendSnapshotTruncate {
     dfs.mkdirs(dir);
     dfs.allowSnapshot(dir);
 
-    final File localDir = new File(
-        System.getProperty("test.build.data", "target/test/data")
-        + dirPathString);
+    final File localDir = GenericTestUtils.getTestDir(dirPathString);
     if (localDir.exists()) {
       FileUtil.fullyDelete(localDir);
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cc6c2651/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 e31de13..24d8b90 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
@@ -3274,8 +3274,8 @@ public class TestDFSShell {
       assertTrue(e.getMessage().contains("Invalid path name /.reserved"));
     }
 
-    final String testdir = System.getProperty("test.build.data")
-        + "/TestDFSShell-testCopyReserved";
+    final String testdir = GenericTestUtils.getTempPath(
+        "TestDFSShell-testCopyReserved");
     final Path hdfsTestDir = new Path(testdir);
     writeFile(fs, new Path(testdir, "testFileForPut"));
     final Path src = new Path(hdfsTestDir, "srcfile");

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cc6c2651/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSUpgradeFromImage.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSUpgradeFromImage.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSUpgradeFromImage.java
index 60bea7a..5dee6e0 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSUpgradeFromImage.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSUpgradeFromImage.java
@@ -89,8 +89,10 @@ public class TestDFSUpgradeFromImage {
   static {
     upgradeConf = new HdfsConfiguration();
     upgradeConf.setInt(DFSConfigKeys.DFS_DATANODE_SCAN_PERIOD_HOURS_KEY, -1); // block scanning off
-    if (System.getProperty("test.build.data") == null) { // to allow test to be run outside of Maven
-      System.setProperty("test.build.data", "build/test/data");
+    if (System.getProperty(GenericTestUtils.SYSPROP_TEST_DATA_DIR) == null) {
+      // to allow test to be run outside of Maven
+      System.setProperty(GenericTestUtils.SYSPROP_TEST_DATA_DIR,
+          GenericTestUtils.DEFAULT_TEST_DATA_DIR);
     }
   }
   
@@ -105,19 +107,19 @@ public class TestDFSUpgradeFromImage {
   
   void unpackStorage(String tarFileName, String referenceName)
       throws IOException {
-    String tarFile = System.getProperty("test.cache.data", "build/test/cache")
+    String tarFile = System.getProperty("test.cache.data", "target/test/cache")
         + "/" + tarFileName;
-    String dataDir = System.getProperty("test.build.data", "build/test/data");
+    File dataDir = GenericTestUtils.getTestDir();
     File dfsDir = new File(dataDir, "dfs");
     if ( dfsDir.exists() && !FileUtil.fullyDelete(dfsDir) ) {
       throw new IOException("Could not delete dfs directory '" + dfsDir + "'");
     }
     LOG.info("Unpacking " + tarFile);
-    FileUtil.unTar(new File(tarFile), new File(dataDir));
+    FileUtil.unTar(new File(tarFile), dataDir);
     //Now read the reference info
     
     BufferedReader reader = new BufferedReader(new FileReader(
-        System.getProperty("test.cache.data", "build/test/cache")
+        System.getProperty("test.cache.data", "target/test/cache")
             + "/" + referenceName));
     String line;
     while ( (line = reader.readLine()) != null ) {
@@ -631,10 +633,10 @@ public class TestDFSUpgradeFromImage {
     unpackStorage(HADOOP1_BBW_IMAGE, HADOOP_DFS_DIR_TXT);
     Configuration conf = new Configuration(upgradeConf);
     conf.set(DFSConfigKeys.DFS_DATANODE_DATA_DIR_KEY, 
-        System.getProperty("test.build.data") + File.separator + 
+        GenericTestUtils.getTempPath(
         "dfs" + File.separator + 
         "data" + File.separator + 
-        "data1");
+        "data1"));
     upgradeAndVerify(new MiniDFSCluster.Builder(conf).
           numDataNodes(1).enableManagedDfsDirsRedundancy(false).
           manageDataDfsDirs(false), null);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cc6c2651/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSUtil.java
index 845f5c8..448ef6a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSUtil.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSUtil.java
@@ -874,8 +874,7 @@ public class TestDFSUtil {
 
   @Test
   public void testGetPassword() throws Exception {
-    File testDir = new File(System.getProperty("test.build.data",
-        "target/test-dir"));
+    File testDir = GenericTestUtils.getTestDir();
 
     Configuration conf = new Configuration();
     final Path jksPath = new Path(testDir.toString(), "test.jks");

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cc6c2651/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDatanodeLayoutUpgrade.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDatanodeLayoutUpgrade.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDatanodeLayoutUpgrade.java
index 6b60b2b..0e2f4e4 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDatanodeLayoutUpgrade.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDatanodeLayoutUpgrade.java
@@ -19,6 +19,7 @@
 package org.apache.hadoop.hdfs;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.test.GenericTestUtils;
 import org.junit.Test;
 
 import java.io.File;
@@ -42,11 +43,11 @@ public class TestDatanodeLayoutUpgrade {
     TestDFSUpgradeFromImage upgrade = new TestDFSUpgradeFromImage();
     upgrade.unpackStorage(HADOOP24_DATANODE, HADOOP_DATANODE_DIR_TXT);
     Configuration conf = new Configuration(TestDFSUpgradeFromImage.upgradeConf);
-    conf.set(DFSConfigKeys.DFS_DATANODE_DATA_DIR_KEY,
-        new File(System.getProperty("test.build.data"),
+    conf.set(
+        DFSConfigKeys.DFS_DATANODE_DATA_DIR_KEY, GenericTestUtils.getTestDir(
             "dfs" + File.separator + "data").toURI().toString());
-    conf.set(DFSConfigKeys.DFS_NAMENODE_NAME_DIR_KEY,
-        new File(System.getProperty("test.build.data"),
+    conf.set(
+        DFSConfigKeys.DFS_NAMENODE_NAME_DIR_KEY, GenericTestUtils.getTestDir(
             "dfs" + File.separator + "name").toURI().toString());
     upgrade.upgradeAndVerify(new MiniDFSCluster.Builder(conf).numDataNodes(1)
     .manageDataDfsDirs(false).manageNameDfsDirs(false), null);
@@ -61,12 +62,12 @@ public class TestDatanodeLayoutUpgrade {
     TestDFSUpgradeFromImage upgrade = new TestDFSUpgradeFromImage();
     upgrade.unpackStorage(HADOOP_56_DN_LAYOUT, HADOOP_56_DN_LAYOUT_TXT);
     Configuration conf = new Configuration(TestDFSUpgradeFromImage.upgradeConf);
-    conf.set(DFSConfigKeys.DFS_DATANODE_DATA_DIR_KEY,
-        new File(System.getProperty("test.build.data"), "dfs" + File.separator
-            + "data").toURI().toString());
-    conf.set(DFSConfigKeys.DFS_NAMENODE_NAME_DIR_KEY,
-        new File(System.getProperty("test.build.data"), "dfs" + File.separator
-            + "name").toURI().toString());
+    conf.set(
+        DFSConfigKeys.DFS_DATANODE_DATA_DIR_KEY, GenericTestUtils.getTestDir(
+            "dfs" + File.separator + "data").toURI().toString());
+    conf.set(
+        DFSConfigKeys.DFS_NAMENODE_NAME_DIR_KEY, GenericTestUtils.getTestDir(
+            "dfs" + File.separator + "name").toURI().toString());
     upgrade.upgradeAndVerify(new MiniDFSCluster.Builder(conf).numDataNodes(1)
         .manageDataDfsDirs(false).manageNameDfsDirs(false), null);
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cc6c2651/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDatanodeStartupFixesLegacyStorageIDs.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDatanodeStartupFixesLegacyStorageIDs.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDatanodeStartupFixesLegacyStorageIDs.java
index e262abc..659a8c1 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDatanodeStartupFixesLegacyStorageIDs.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDatanodeStartupFixesLegacyStorageIDs.java
@@ -87,12 +87,12 @@ public class TestDatanodeStartupFixesLegacyStorageIDs {
 
   private static void initStorageDirs(final Configuration conf,
                                       final String testName) {
-    conf.set(DFSConfigKeys.DFS_DATANODE_DATA_DIR_KEY,
-             System.getProperty("test.build.data") + File.separator +
-                 testName + File.separator + "dfs" + File.separator + "data");
-    conf.set(DFSConfigKeys.DFS_NAMENODE_NAME_DIR_KEY,
-             System.getProperty("test.build.data") + File.separator +
-                 testName + File.separator + "dfs" + File.separator + "name");
+    conf.set(
+        DFSConfigKeys.DFS_DATANODE_DATA_DIR_KEY, GenericTestUtils.getTempPath(
+            testName + File.separator + "dfs" + File.separator + "data"));
+    conf.set(
+        DFSConfigKeys.DFS_NAMENODE_NAME_DIR_KEY, GenericTestUtils.getTempPath(
+            testName + File.separator + "dfs" + File.separator + "name"));
 
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cc6c2651/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFetchImage.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFetchImage.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFetchImage.java
index 6f61003..d8218b6 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFetchImage.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFetchImage.java
@@ -32,13 +32,14 @@ import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
 import org.apache.hadoop.hdfs.tools.DFSAdmin;
 import org.apache.hadoop.hdfs.util.MD5FileUtils;
 import org.apache.hadoop.io.MD5Hash;
+import org.apache.hadoop.test.GenericTestUtils;
 import org.junit.AfterClass;
 import org.junit.Test;
 
 public class TestFetchImage {
   
-  private static final File FETCHED_IMAGE_FILE = new File(
-      System.getProperty("test.build.dir"), "target/fetched-image-dir");
+  private static final File FETCHED_IMAGE_FILE =
+      GenericTestUtils.getTestDir("target/fetched-image-dir");
   // Shamelessly stolen from NNStorage.
   private static final Pattern IMAGE_REGEX = Pattern.compile("fsimage_(\\d+)");
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cc6c2651/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestPread.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestPread.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestPread.java
index 2789e33..c761225 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestPread.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestPread.java
@@ -482,7 +482,7 @@ public class TestPread {
     Configuration conf = new HdfsConfiguration();
     FileSystem fileSys = FileSystem.getLocal(conf);
     try {
-      Path file1 = new Path("build/test/data", "preadtest.dat");
+      Path file1 = new Path(GenericTestUtils.getTempPath("preadtest.dat"));
       writeFile(fileSys, file1);
       pReadFile(fileSys, file1);
       cleanupFile(fileSys, file1);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cc6c2651/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestSeekBug.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestSeekBug.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestSeekBug.java
index c9f5293..9dd2987 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestSeekBug.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestSeekBug.java
@@ -30,6 +30,7 @@ import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.IOUtils;
+import org.apache.hadoop.test.GenericTestUtils;
 import org.junit.Test;
 
 /**
@@ -204,7 +205,7 @@ public class TestSeekBug {
     Configuration conf = new HdfsConfiguration();
     FileSystem fileSys = FileSystem.getLocal(conf);
     try {
-      Path file1 = new Path("build/test/data", "seektest.dat");
+      Path file1 = new Path(GenericTestUtils.getTempPath("seektest.dat"));
       DFSTestUtil.createFile(fileSys, file1, ONEMB, ONEMB,
           fileSys.getDefaultBlockSize(file1),
           fileSys.getDefaultReplication(file1), seed);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cc6c2651/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/protocol/datatransfer/sasl/SaslDataTransferTestCase.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/protocol/datatransfer/sasl/SaslDataTransferTestCase.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/protocol/datatransfer/sasl/SaslDataTransferTestCase.java
index 37b2a2c..4eade6a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/protocol/datatransfer/sasl/SaslDataTransferTestCase.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/protocol/datatransfer/sasl/SaslDataTransferTestCase.java
@@ -43,6 +43,7 @@ import org.apache.hadoop.minikdc.MiniKdc;
 import org.apache.hadoop.security.SecurityUtil;
 import org.apache.hadoop.security.UserGroupInformation.AuthenticationMethod;
 import org.apache.hadoop.security.ssl.KeyStoreTestUtil;
+import org.apache.hadoop.test.GenericTestUtils;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 
@@ -74,8 +75,8 @@ public abstract class SaslDataTransferTestCase {
 
   @BeforeClass
   public static void initKdc() throws Exception {
-    baseDir = new File(System.getProperty("test.build.dir", "target/test-dir"),
-      SaslDataTransferTestCase.class.getSimpleName());
+    baseDir = GenericTestUtils
+        .getTestDir(SaslDataTransferTestCase.class.getSimpleName());
     FileUtil.fullyDelete(baseDir);
     assertTrue(baseDir.mkdirs());
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cc6c2651/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/TestSecureNNWithQJM.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/TestSecureNNWithQJM.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/TestSecureNNWithQJM.java
index 197759e..9abfb9c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/TestSecureNNWithQJM.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/TestSecureNNWithQJM.java
@@ -55,6 +55,7 @@ import org.apache.hadoop.security.SecurityUtil;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.UserGroupInformation.AuthenticationMethod;
 import org.apache.hadoop.security.ssl.KeyStoreTestUtil;
+import org.apache.hadoop.test.GenericTestUtils;
 import org.junit.After;
 import org.junit.AfterClass;
 import org.junit.Before;
@@ -82,8 +83,8 @@ public class TestSecureNNWithQJM {
 
   @BeforeClass
   public static void init() throws Exception {
-    baseDir = new File(System.getProperty("test.build.dir", "target/test-dir"),
-      TestSecureNNWithQJM.class.getSimpleName());
+    baseDir =
+        GenericTestUtils.getTestDir(TestSecureNNWithQJM.class.getSimpleName());
     FileUtil.fullyDelete(baseDir);
     assertTrue(baseDir.mkdirs());
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cc6c2651/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/balancer/TestBalancer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/balancer/TestBalancer.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/balancer/TestBalancer.java
index 2ad8edd..738cfe6 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/balancer/TestBalancer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/balancer/TestBalancer.java
@@ -211,8 +211,7 @@ public class TestBalancer {
   }
 
   static void initSecureConf(Configuration conf) throws Exception {
-    baseDir = new File(System.getProperty("test.build.dir", "target/test-dir"),
-        TestBalancer.class.getSimpleName());
+    baseDir = GenericTestUtils.getTestDir(TestBalancer.class.getSimpleName());
     FileUtil.fullyDelete(baseDir);
     assertTrue(baseDir.mkdirs());
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cc6c2651/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeUUID.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeUUID.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeUUID.java
index ebf7c35..0d40fc7 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeUUID.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeUUID.java
@@ -25,6 +25,7 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.test.GenericTestUtils;
 import org.junit.Test;
 
 import java.io.File;
@@ -69,7 +70,7 @@ public class TestDataNodeUUID {
 
   @Test(timeout = 10000)
   public void testUUIDRegeneration() throws Exception {
-    File baseDir = new File(System.getProperty("test.build.data"));
+    File baseDir = GenericTestUtils.getTestDir();
     File disk1 = new File(baseDir, "disk1");
     File disk2 = new File(baseDir, "disk2");
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cc6c2651/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataStorage.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataStorage.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataStorage.java
index 405d2e9..446a77b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataStorage.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataStorage.java
@@ -47,8 +47,7 @@ public class TestDataStorage {
   private final static String BUILD_VERSION = "2.0";
   private final static String SOFTWARE_VERSION = "2.0";
   private final static long CTIME = 1;
-  private final static File TEST_DIR =
-      new File(System.getProperty("test.build.data") + "/dstest");
+  private final static File TEST_DIR = GenericTestUtils.getTestDir("dstest");
   private final static StartupOption START_OPT = StartupOption.REGULAR;
 
   private DataNode mockDN = Mockito.mock(DataNode.class);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cc6c2651/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCreateEditsLog.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCreateEditsLog.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCreateEditsLog.java
index 67e5e33..d3527f5 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCreateEditsLog.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCreateEditsLog.java
@@ -22,6 +22,7 @@ import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_NAME_DIR_KEY;
 
 import java.io.File;
 
+import org.apache.hadoop.test.GenericTestUtils;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
@@ -42,9 +43,8 @@ public class TestCreateEditsLog {
 
   private static final File HDFS_DIR = new File(
     MiniDFSCluster.getBaseDirectory()).getAbsoluteFile();
-  private static final File TEST_DIR = new File(
-    System.getProperty("test.build.data", "build/test/data"),
-    "TestCreateEditsLog").getAbsoluteFile();
+  private static final File TEST_DIR =
+      GenericTestUtils.getTestDir("TestCreateEditsLog");
 
   private MiniDFSCluster cluster;
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cc6c2651/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestDecommissioningStatus.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestDecommissioningStatus.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestDecommissioningStatus.java
index 341933e..7c39bf8 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestDecommissioningStatus.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestDecommissioningStatus.java
@@ -25,7 +25,6 @@ import java.io.PrintStream;
 import java.net.InetSocketAddress;
 import java.util.ArrayList;
 import java.util.Arrays;
-import java.util.Iterator;
 import java.util.List;
 import java.util.Random;
 
@@ -55,6 +54,7 @@ import org.apache.hadoop.hdfs.server.blockmanagement.DecommissionManager;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
 import org.apache.hadoop.hdfs.server.datanode.DataNodeTestUtils;
 import org.apache.hadoop.hdfs.tools.DFSAdmin;
+import org.apache.hadoop.hdfs.util.HostsFileWriter;
 import org.apache.log4j.Level;
 import org.apache.log4j.Logger;
 import org.junit.After;
@@ -71,10 +71,8 @@ public class TestDecommissioningStatus {
   private static final int numDatanodes = 2;
   private static MiniDFSCluster cluster;
   private static FileSystem fileSys;
-  private static Path excludeFile;
-  private static FileSystem localFileSys;
+  private static HostsFileWriter hostsFileWriter;
   private static Configuration conf;
-  private static Path dir;
 
   final ArrayList<String> decommissionedNodes = new ArrayList<String>(numDatanodes);
   
@@ -85,14 +83,8 @@ public class TestDecommissioningStatus {
         false);
 
     // Set up the hosts/exclude files.
-    localFileSys = FileSystem.getLocal(conf);
-    Path workingDir = localFileSys.getWorkingDirectory();
-    dir = new Path(workingDir, "build/test/data/work-dir/decommission");
-    assertTrue(localFileSys.mkdirs(dir));
-    excludeFile = new Path(dir, "exclude");
-    conf.set(DFSConfigKeys.DFS_HOSTS_EXCLUDE, excludeFile.toUri().getPath());
-    Path includeFile = new Path(dir, "include");
-    conf.set(DFSConfigKeys.DFS_HOSTS, includeFile.toUri().getPath());
+    hostsFileWriter = new HostsFileWriter();
+    hostsFileWriter.initialize(conf, "work-dir/decommission");
     conf.setInt(DFSConfigKeys.DFS_NAMENODE_HEARTBEAT_RECHECK_INTERVAL_KEY, 
         1000);
     conf.setInt(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY, 1);
@@ -102,9 +94,6 @@ public class TestDecommissioningStatus {
     conf.setInt(DFSConfigKeys.DFS_NAMENODE_DECOMMISSION_INTERVAL_KEY, 1);
     conf.setLong(DFSConfigKeys.DFS_DATANODE_BALANCE_BANDWIDTHPERSEC_KEY, 1);
 
-    writeConfigFile(localFileSys, excludeFile, null);
-    writeConfigFile(localFileSys, includeFile, null);
-
     cluster = new MiniDFSCluster.Builder(conf).numDataNodes(numDatanodes).build();
     cluster.waitActive();
     fileSys = cluster.getFileSystem();
@@ -115,31 +104,13 @@ public class TestDecommissioningStatus {
 
   @After
   public void tearDown() throws Exception {
-    if (localFileSys != null ) cleanupFile(localFileSys, dir);
+    if (hostsFileWriter != null) {
+      hostsFileWriter.cleanup();
+    }
     if(fileSys != null) fileSys.close();
     if(cluster != null) cluster.shutdown();
   }
 
-  private static void writeConfigFile(FileSystem fs, Path name,
-      ArrayList<String> nodes) throws IOException {
-
-    // delete if it already exists
-    if (fs.exists(name)) {
-      fs.delete(name, true);
-    }
-
-    FSDataOutputStream stm = fs.create(name);
-
-    if (nodes != null) {
-      for (Iterator<String> it = nodes.iterator(); it.hasNext();) {
-        String node = it.next();
-        stm.writeBytes(node);
-        stm.writeBytes("\n");
-      }
-    }
-    stm.close();
-  }
-
   private FSDataOutputStream writeIncompleteFile(FileSystem fileSys, Path name,
       short repl) throws IOException {
     // create and write a file that contains three blocks of data
@@ -169,25 +140,25 @@ public class TestDecommissioningStatus {
    * Decommissions the node at the given index
    */
   private String decommissionNode(FSNamesystem namesystem, DFSClient client,
-      FileSystem localFileSys, int nodeIndex) throws IOException {
+      int nodeIndex) throws IOException {
     DatanodeInfo[] info = client.datanodeReport(DatanodeReportType.LIVE);
 
     String nodename = info[nodeIndex].getXferAddr();
-    decommissionNode(namesystem, localFileSys, nodename);
+    decommissionNode(namesystem, nodename);
     return nodename;
   }
 
   /*
    * Decommissions the node by name
    */
-  private void decommissionNode(FSNamesystem namesystem,
-      FileSystem localFileSys, String dnName) throws IOException {
+  private void decommissionNode(FSNamesystem namesystem, String dnName)
+      throws IOException {
     System.out.println("Decommissioning node: " + dnName);
 
     // write nodename into the exclude file.
     ArrayList<String> nodes = new ArrayList<String>(decommissionedNodes);
     nodes.add(dnName);
-    writeConfigFile(localFileSys, excludeFile, nodes);
+    hostsFileWriter.initExcludeHosts(nodes.toArray(new String[0]));
   }
 
   private void checkDecommissionStatus(DatanodeDescriptor decommNode,
@@ -280,7 +251,7 @@ public class TestDecommissioningStatus {
     FSNamesystem fsn = cluster.getNamesystem();
     final DatanodeManager dm = fsn.getBlockManager().getDatanodeManager();
     for (int iteration = 0; iteration < numDatanodes; iteration++) {
-      String downnode = decommissionNode(fsn, client, localFileSys, iteration);
+      String downnode = decommissionNode(fsn, client, iteration);
       dm.refreshNodes(conf);
       decommissionedNodes.add(downnode);
       BlockManagerTestUtil.recheckDecommissionState(dm);
@@ -307,7 +278,7 @@ public class TestDecommissioningStatus {
     // Call refreshNodes on FSNamesystem with empty exclude file.
     // This will remove the datanodes from decommissioning list and
     // make them available again.
-    writeConfigFile(localFileSys, excludeFile, null);
+    hostsFileWriter.initExcludeHost("");
     dm.refreshNodes(conf);
     st1.close();
     cleanupFile(fileSys, file1);
@@ -337,7 +308,7 @@ public class TestDecommissioningStatus {
     // Decommission the DN.
     FSNamesystem fsn = cluster.getNamesystem();
     final DatanodeManager dm = fsn.getBlockManager().getDatanodeManager();
-    decommissionNode(fsn, localFileSys, dnName);
+    decommissionNode(fsn, dnName);
     dm.refreshNodes(conf);
 
     // Stop the DN when decommission is in progress.
@@ -384,7 +355,7 @@ public class TestDecommissioningStatus {
     // Call refreshNodes on FSNamesystem with empty exclude file.
     // This will remove the datanodes from decommissioning list and
     // make them available again.
-    writeConfigFile(localFileSys, excludeFile, null);
+    hostsFileWriter.initExcludeHost("");
     dm.refreshNodes(conf);
   }
 
@@ -405,7 +376,7 @@ public class TestDecommissioningStatus {
     FSNamesystem fsn = cluster.getNamesystem();
     final DatanodeManager dm = fsn.getBlockManager().getDatanodeManager();
     DatanodeDescriptor dnDescriptor = dm.getDatanode(dnID);
-    decommissionNode(fsn, localFileSys, dnName);
+    decommissionNode(fsn, dnName);
     dm.refreshNodes(conf);
     BlockManagerTestUtil.recheckDecommissionState(dm);
     assertTrue(dnDescriptor.isDecommissioned());
@@ -416,7 +387,7 @@ public class TestDecommissioningStatus {
 
     // Call refreshNodes on FSNamesystem with empty exclude file to remove the
     // datanode from decommissioning list and make it available again.
-    writeConfigFile(localFileSys, excludeFile, null);
+    hostsFileWriter.initExcludeHost("");
     dm.refreshNodes(conf);
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cc6c2651/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestEditLogFileInputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestEditLogFileInputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestEditLogFileInputStream.java
index aecdc78..cd329a6 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestEditLogFileInputStream.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestEditLogFileInputStream.java
@@ -87,8 +87,7 @@ public class TestEditLogFileInputStream {
   @Test(timeout=60000)
   public void testScanCorruptEditLog() throws Exception {
     Configuration conf = new Configuration();
-    File editLog = new File(System.getProperty(
-        "test.build.data", "/tmp"), "testCorruptEditLog");
+    File editLog = new File(GenericTestUtils.getTempPath("testCorruptEditLog"));
 
     LOG.debug("Creating test edit log file: " + editLog);
     EditLogFileOutputStream elos = new EditLogFileOutputStream(conf,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cc6c2651/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSImageWithSnapshot.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSImageWithSnapshot.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSImageWithSnapshot.java
index 6be3950..82f5cfb 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSImageWithSnapshot.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSImageWithSnapshot.java
@@ -69,8 +69,8 @@ public class TestFSImageWithSnapshot {
 
   private final Path dir = new Path("/TestSnapshot");
   private static final String testDir =
-      System.getProperty("test.build.data", "build/test/data");
-  
+      GenericTestUtils.getTestDir().getAbsolutePath();
+
   Configuration conf;
   MiniDFSCluster cluster;
   FSNamesystem fsn;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cc6c2651/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 73ad885..859f72c 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
@@ -119,8 +119,8 @@ import com.google.common.collect.Sets;
  * A JUnit test for doing fsck
  */
 public class TestFsck {
-  static final String auditLogFile = System.getProperty("test.build.dir",
-      "build/test") + "/TestFsck-audit.log";
+  static final String AUDITLOG_FILE =
+      GenericTestUtils.getTempPath("TestFsck-audit.log");
   
   // Pattern for: 
   // allowed=true ugi=name ip=/address cmd=FSCK src=/ dst=null perm=null
@@ -212,14 +212,15 @@ public class TestFsck {
 
   /** Sets up log4j logger for auditlogs */
   private void setupAuditLogs() throws IOException {
-    File file = new File(auditLogFile);
+    File file = new File(AUDITLOG_FILE);
     if (file.exists()) {
       file.delete();
     }
     Logger logger = ((Log4JLogger) FSNamesystem.auditLog).getLogger();
     logger.setLevel(Level.INFO);
     PatternLayout layout = new PatternLayout("%m%n");
-    RollingFileAppender appender = new RollingFileAppender(layout, auditLogFile);
+    RollingFileAppender appender =
+        new RollingFileAppender(layout, AUDITLOG_FILE);
     logger.addAppender(appender);
   }
   
@@ -231,7 +232,7 @@ public class TestFsck {
     BufferedReader reader = null;
     try {
       // Audit log should contain one getfileinfo and one fsck
-      reader = new BufferedReader(new FileReader(auditLogFile));
+      reader = new BufferedReader(new FileReader(AUDITLOG_FILE));
       String line;
 
       // one extra getfileinfo stems from resolving the path

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cc6c2651/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeHttpServer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeHttpServer.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeHttpServer.java
index 7dc719e..27efea6 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeHttpServer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeHttpServer.java
@@ -32,6 +32,7 @@ import org.apache.hadoop.http.HttpConfig;
 import org.apache.hadoop.http.HttpConfig.Policy;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.security.ssl.KeyStoreTestUtil;
+import org.apache.hadoop.test.GenericTestUtils;
 import org.junit.AfterClass;
 import org.junit.Assert;
 import org.junit.BeforeClass;
@@ -42,8 +43,8 @@ import org.junit.runners.Parameterized.Parameters;
 
 @RunWith(value = Parameterized.class)
 public class TestNameNodeHttpServer {
-  private static final String BASEDIR = System.getProperty("test.build.dir",
-      "target/test-dir") + "/" + TestNameNodeHttpServer.class.getSimpleName();
+  private static final String BASEDIR = GenericTestUtils
+      .getTempPath(TestNameNodeHttpServer.class.getSimpleName());
   private static String keystoresDir;
   private static String sslConfDir;
   private static Configuration conf;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cc6c2651/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeRecovery.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeRecovery.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeRecovery.java
index 87e2523..877f43c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeRecovery.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeRecovery.java
@@ -49,6 +49,7 @@ import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.DeleteOp;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.OpInstanceCache;
 import org.apache.hadoop.hdfs.server.namenode.NNStorage.NameNodeDirType;
 import org.apache.hadoop.io.IOUtils;
+import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.test.PathUtils;
 import org.apache.hadoop.util.StringUtils;
 import org.junit.Test;
@@ -521,8 +522,8 @@ public class TestNameNodeRecovery {
     conf.set(DFSConfigKeys.DFS_HA_NAMENODE_ID_KEY, "nn1");
     conf.set(DFSUtil.addKeySuffixes(DFSConfigKeys.DFS_HA_NAMENODES_KEY_PREFIX,
       "ns1"), "nn1,nn2");
-    String baseDir = System.getProperty(
-        MiniDFSCluster.PROP_TEST_BUILD_DATA, "build/test/data") + "/dfs/";
+    String baseDir = GenericTestUtils.getTestDir("setupRecoveryTestConf")
+        .getAbsolutePath();
     File nameDir = new File(baseDir, "nameR");
     File secondaryDir = new File(baseDir, "namesecondaryR");
     conf.set(DFSUtil.addKeySuffixes(DFSConfigKeys.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cc6c2651/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeRespectsBindHostKeys.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeRespectsBindHostKeys.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeRespectsBindHostKeys.java
index 0c65360..21d99a3 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeRespectsBindHostKeys.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeRespectsBindHostKeys.java
@@ -28,6 +28,7 @@ import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.http.HttpConfig;
 import org.apache.hadoop.security.ssl.KeyStoreTestUtil;
+import org.apache.hadoop.test.GenericTestUtils;
 
 import java.io.File;
 import java.io.IOException;
@@ -237,8 +238,8 @@ public class TestNameNodeRespectsBindHostKeys {
     }
   }
 
-  private static final String BASEDIR = System.getProperty("test.build.dir",
-      "target/test-dir") + "/" + TestNameNodeRespectsBindHostKeys.class.getSimpleName();
+  private static final String BASEDIR = GenericTestUtils
+      .getTempPath(TestNameNodeRespectsBindHostKeys.class.getSimpleName());
 
   private static void setupSsl() throws Exception {
     Configuration conf = new Configuration();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cc6c2651/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestRenameWithSnapshots.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestRenameWithSnapshots.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestRenameWithSnapshots.java
index 827feb6..ad3a5a1 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestRenameWithSnapshots.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestRenameWithSnapshots.java
@@ -95,7 +95,7 @@ public class TestRenameWithSnapshots {
   private static FSDirectory fsdir;
   private static DistributedFileSystem hdfs;
   private static final String testDir =
-      System.getProperty("test.build.data", "build/test/data");
+      GenericTestUtils.getTestDir().getAbsolutePath();
   static private final Path dir = new Path("/testRenameWithSnapshots");
   static private final Path sub1 = new Path(dir, "sub1");
   static private final Path file1 = new Path(sub1, "file1");

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cc6c2651/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSnapshot.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSnapshot.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSnapshot.java
index 6313ce1..4a8fdc8 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSnapshot.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSnapshot.java
@@ -98,8 +98,8 @@ public class TestSnapshot {
   protected DistributedFileSystem hdfs;
   
   private static final String testDir =
-      System.getProperty("test.build.data", "build/test/data");
-  
+      GenericTestUtils.getTestDir().getAbsolutePath();
+
   @Rule
   public ExpectedException exception = ExpectedException.none();
   

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cc6c2651/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/TestGetConf.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/TestGetConf.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/TestGetConf.java
index 94ce6b2..942719e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/TestGetConf.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/TestGetConf.java
@@ -35,20 +35,18 @@ import java.io.PrintStream;
 import java.net.InetSocketAddress;
 import java.util.ArrayList;
 import java.util.Arrays;
-import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 import java.util.StringTokenizer;
 
-import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.DFSUtil.ConfiguredNNAddress;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.tools.GetConf.Command;
 import org.apache.hadoop.hdfs.tools.GetConf.CommandHandler;
+import org.apache.hadoop.hdfs.util.HostsFileWriter;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.util.ToolRunner;
 import org.junit.Test;
@@ -390,42 +388,29 @@ public class TestGetConf {
   public void TestGetConfExcludeCommand() throws Exception{
   	HdfsConfiguration conf = new HdfsConfiguration();
     // Set up the hosts/exclude files.
-    localFileSys = FileSystem.getLocal(conf);
-    Path workingDir = localFileSys.getWorkingDirectory();
-    Path dir = new Path(workingDir, System.getProperty("test.build.data", "target/test/data") + "/Getconf/");
-    Path hostsFile = new Path(dir, "hosts");
-    Path excludeFile = new Path(dir, "exclude");
-    
-    // Setup conf
-    conf.set(DFSConfigKeys.DFS_HOSTS, hostsFile.toUri().getPath());
-    conf.set(DFSConfigKeys.DFS_HOSTS_EXCLUDE, excludeFile.toUri().getPath());
-    writeConfigFile(hostsFile, null);
-    writeConfigFile(excludeFile, null);    
+    HostsFileWriter hostsFileWriter = new HostsFileWriter();
+    hostsFileWriter.initialize(conf, "GetConf");
+    Path excludeFile = hostsFileWriter.getExcludeFile();
+
     String[] args = {"-excludeFile"};
     String ret = runTool(conf, args, true);
     assertEquals(excludeFile.toUri().getPath(),ret.trim());
-    cleanupFile(localFileSys, excludeFile.getParent());
+    hostsFileWriter.cleanup();
   }
   
   @Test
   public void TestGetConfIncludeCommand() throws Exception{
   	HdfsConfiguration conf = new HdfsConfiguration();
     // Set up the hosts/exclude files.
-    localFileSys = FileSystem.getLocal(conf);
-    Path workingDir = localFileSys.getWorkingDirectory();
-    Path dir = new Path(workingDir, System.getProperty("test.build.data", "target/test/data") + "/Getconf/");
-    Path hostsFile = new Path(dir, "hosts");
-    Path excludeFile = new Path(dir, "exclude");
-    
+    HostsFileWriter hostsFileWriter = new HostsFileWriter();
+    hostsFileWriter.initialize(conf, "GetConf");
+    Path hostsFile = hostsFileWriter.getIncludeFile();
+
     // Setup conf
-    conf.set(DFSConfigKeys.DFS_HOSTS, hostsFile.toUri().getPath());
-    conf.set(DFSConfigKeys.DFS_HOSTS_EXCLUDE, excludeFile.toUri().getPath());
-    writeConfigFile(hostsFile, null);
-    writeConfigFile(excludeFile, null);    
     String[] args = {"-includeFile"};
     String ret = runTool(conf, args, true);
     assertEquals(hostsFile.toUri().getPath(),ret.trim());
-    cleanupFile(localFileSys, excludeFile.getParent());
+    hostsFileWriter.cleanup();
   }
 
   @Test
@@ -443,29 +428,4 @@ public class TestGetConf {
     verifyAddresses(conf, TestType.NAMENODE, false, includedNN);
     verifyAddresses(conf, TestType.NNRPCADDRESSES, true, includedNN);
   }
-
-  private void writeConfigFile(Path name, ArrayList<String> nodes) 
-      throws IOException {
-      // delete if it already exists
-      if (localFileSys.exists(name)) {
-        localFileSys.delete(name, true);
-      }
-
-      FSDataOutputStream stm = localFileSys.create(name);
-      
-      if (nodes != null) {
-        for (Iterator<String> it = nodes.iterator(); it.hasNext();) {
-          String node = it.next();
-          stm.writeBytes(node);
-          stm.writeBytes("\n");
-        }
-      }
-      stm.close();
-    }
-  
-  private void cleanupFile(FileSystem fileSys, Path name) throws IOException {
-    assertTrue(fileSys.exists(name));
-    fileSys.delete(name, true);
-    assertTrue(!fileSys.exists(name));
-  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cc6c2651/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/util/HostsFileWriter.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/util/HostsFileWriter.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/util/HostsFileWriter.java
index cd5ae95..2ef0b8f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/util/HostsFileWriter.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/util/HostsFileWriter.java
@@ -73,16 +73,26 @@ public class HostsFileWriter {
   }
 
   public void initExcludeHost(String hostNameAndPort) throws IOException {
+    initExcludeHosts(hostNameAndPort);
+  }
+
+  public void initExcludeHosts(String... hostNameAndPorts) throws IOException {
+    StringBuilder excludeHosts = new StringBuilder();
     if (isLegacyHostsFile) {
-      DFSTestUtil.writeFile(localFileSys, excludeFile, hostNameAndPort);
+      for (String hostNameAndPort : hostNameAndPorts) {
+        excludeHosts.append(hostNameAndPort).append("\n");
+      }
+      DFSTestUtil.writeFile(localFileSys, excludeFile, excludeHosts.toString());
     } else {
-      DatanodeAdminProperties dn = new DatanodeAdminProperties();
-      String [] hostAndPort = hostNameAndPort.split(":");
-      dn.setHostName(hostAndPort[0]);
-      dn.setPort(Integer.parseInt(hostAndPort[1]));
-      dn.setAdminState(AdminStates.DECOMMISSIONED);
       HashSet<DatanodeAdminProperties> allDNs = new HashSet<>();
-      allDNs.add(dn);
+      for (String hostNameAndPort : hostNameAndPorts) {
+        DatanodeAdminProperties dn = new DatanodeAdminProperties();
+        String[] hostAndPort = hostNameAndPort.split(":");
+        dn.setHostName(hostAndPort[0]);
+        dn.setPort(Integer.parseInt(hostAndPort[1]));
+        dn.setAdminState(AdminStates.DECOMMISSIONED);
+        allDNs.add(dn);
+      }
       CombinedHostsFileWriter.writeFile(combinedFile.toString(), allDNs);
     }
   }
@@ -119,4 +129,12 @@ public class HostsFileWriter {
       FileUtils.deleteQuietly(new File(fullDir.toUri().getPath()));
     }
   }
+
+  public Path getIncludeFile() {
+    return includeFile;
+  }
+
+  public Path getExcludeFile() {
+    return excludeFile;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cc6c2651/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/util/TestCombinedHostsFileReader.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/util/TestCombinedHostsFileReader.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/util/TestCombinedHostsFileReader.java
index c3946e4..923cf66 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/util/TestCombinedHostsFileReader.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/util/TestCombinedHostsFileReader.java
@@ -23,6 +23,7 @@ import java.io.FileWriter;
 import java.util.Set;
 
 import org.apache.hadoop.hdfs.protocol.DatanodeAdminProperties;
+import org.apache.hadoop.test.GenericTestUtils;
 import org.junit.Before;
 import org.junit.After;
 import org.junit.Test;
@@ -35,8 +36,8 @@ import static org.junit.Assert.assertEquals;
 public class TestCombinedHostsFileReader {
 
   // Using /test/build/data/tmp directory to store temporary files
-  static final String HOSTS_TEST_DIR = new File(System.getProperty(
-      "test.build.data", "/tmp")).getAbsolutePath();
+  static final String HOSTS_TEST_DIR = GenericTestUtils.getTestDir()
+      .getAbsolutePath();
   File NEW_FILE = new File(HOSTS_TEST_DIR, "dfs.hosts.new.json");
 
   static final String TEST_CACHE_DATA_DIR =

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cc6c2651/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestHttpsFileSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestHttpsFileSystem.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestHttpsFileSystem.java
index 0f0ac3b..34bb336 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestHttpsFileSystem.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestHttpsFileSystem.java
@@ -32,14 +32,15 @@ import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.http.HttpConfig;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.security.ssl.KeyStoreTestUtil;
+import org.apache.hadoop.test.GenericTestUtils;
 import org.junit.AfterClass;
 import org.junit.Assert;
 import org.junit.BeforeClass;
 import org.junit.Test;
 
 public class TestHttpsFileSystem {
-  private static final String BASEDIR = System.getProperty("test.build.dir",
-      "target/test-dir") + "/" + TestHttpsFileSystem.class.getSimpleName();
+  private static final String BASEDIR =
+      GenericTestUtils.getTempPath(TestHttpsFileSystem.class.getSimpleName());
 
   private static MiniDFSCluster cluster;
   private static Configuration conf;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cc6c2651/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHdfsTokens.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHdfsTokens.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHdfsTokens.java
index cb1efae..6192ad9 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHdfsTokens.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHdfsTokens.java
@@ -55,6 +55,7 @@ import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.authentication.client.ConnectionConfigurator;
 import org.apache.hadoop.security.ssl.KeyStoreTestUtil;
 import org.apache.hadoop.security.token.SecretManager.InvalidToken;
+import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.security.token.Token;
 import org.junit.Assert;
 import org.junit.BeforeClass;
@@ -199,8 +200,8 @@ public class TestWebHdfsTokens {
       SecurityUtil.setAuthenticationMethod(SIMPLE, clusterConf);
       clusterConf.setBoolean(DFSConfigKeys
 	    .DFS_NAMENODE_DELEGATION_TOKEN_ALWAYS_USE_KEY, true);
-      String BASEDIR = System.getProperty("test.build.dir",
-	      	  "target/test-dir") + "/" + TestWebHdfsTokens.class.getSimpleName();
+      String baseDir =
+          GenericTestUtils.getTempPath(TestWebHdfsTokens.class.getSimpleName());
       String keystoresDir;
       String sslConfDir;
 	    
@@ -208,10 +209,10 @@ public class TestWebHdfsTokens {
       clusterConf.set(DFSConfigKeys.DFS_NAMENODE_HTTPS_ADDRESS_KEY, "localhost:0");
       clusterConf.set(DFSConfigKeys.DFS_DATANODE_HTTPS_ADDRESS_KEY, "localhost:0");
 	  
-      File base = new File(BASEDIR);
+      File base = new File(baseDir);
       FileUtil.fullyDelete(base);
       base.mkdirs();
-      keystoresDir = new File(BASEDIR).getAbsolutePath();
+      keystoresDir = new File(baseDir).getAbsolutePath();
       sslConfDir = KeyStoreTestUtil.getClasspathDir(TestWebHdfsTokens.class);
       KeyStoreTestUtil.setupSSLConfig(keystoresDir, sslConfDir, clusterConf, false);
       clusterConf.set(DFSConfigKeys.DFS_CLIENT_HTTPS_KEYSTORE_RESOURCE_KEY,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cc6c2651/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/test/PathUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/test/PathUtils.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/test/PathUtils.java
index ac0f632..d54ea65 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/test/PathUtils.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/test/PathUtils.java
@@ -19,7 +19,6 @@ package org.apache.hadoop.test;
 
 import java.io.File;
 
-import org.apache.commons.lang.RandomStringUtils;
 import org.apache.hadoop.fs.Path;
 
 public class PathUtils {
@@ -37,10 +36,8 @@ public class PathUtils {
   }
   
   public static File getTestDir(Class<?> caller, boolean create) {
-    File dir =
-        new File(System.getProperty("test.build.data", "target/test/data")
-            + "/" + RandomStringUtils.randomAlphanumeric(10),
-            caller.getSimpleName());
+    File dir = new File(GenericTestUtils.getRandomizedTestDir(),
+        caller.getSimpleName());
     if (create) {
       dir.mkdirs();
     }


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


[03/39] hadoop git commit: Revert "HADOOP-12893. Verify LICENSE.txt and NOTICE.txt. Contributed by Xiao Chen, Akira Ajisaka, and Andrew Wang."

Posted by ae...@apache.org.
Revert "HADOOP-12893. Verify LICENSE.txt and NOTICE.txt. Contributed by Xiao Chen, Akira Ajisaka, and Andrew Wang."

This reverts commit e383b732c54c542482b0b836e2d2c46eb49b4e2d.


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

Branch: refs/heads/HDFS-1312
Commit: ee55b74fb93ee3fbe7da8c5f9e3bdeb23bdc9d67
Parents: eec835e
Author: Akira Ajisaka <aa...@apache.org>
Authored: Tue Jun 14 14:26:25 2016 +0900
Committer: Akira Ajisaka <aa...@apache.org>
Committed: Tue Jun 14 14:26:52 2016 +0900

----------------------------------------------------------------------
 LICENSE.txt                 | 1017 +-------------------------------------
 NOTICE.txt                  |  266 ----------
 hadoop-build-tools/pom.xml  |   41 --
 hadoop-project-dist/pom.xml |    2 -
 hadoop-project/pom.xml      |   19 +-
 pom.xml                     |   26 -
 6 files changed, 4 insertions(+), 1367 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/ee55b74f/LICENSE.txt
----------------------------------------------------------------------
diff --git a/LICENSE.txt b/LICENSE.txt
index 44880df..929e2a8 100644
--- a/LICENSE.txt
+++ b/LICENSE.txt
@@ -320,9 +320,7 @@ THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
 (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
 OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
 
-The binary distribution of this product bundles these dependencies under the
-following license:
-re2j 1.0
+For com.google.re2j.* classes:
 ---------------------------------------------------------------------
 This is a work derived from Russ Cox's RE2 in Go, whose license
 http://golang.org/LICENSE is as follows:
@@ -550,14 +548,12 @@ hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/static/bootstrap-3.0.2
 hadoop-tools/hadoop-sls/src/main/html/js/thirdparty/bootstrap.min.js
 hadoop-tools/hadoop-sls/src/main/html/css/bootstrap.min.css
 hadoop-tools/hadoop-sls/src/main/html/css/bootstrap-responsive.min.css
-And the binary distribution of this product bundles these dependencies under the
-following license:
-Mockito 1.8.5
-SLF4J 1.7.10
 --------------------------------------------------------------------------------
 
 The MIT License (MIT)
 
+Copyright (c) 2011-2016 Twitter, Inc.
+
 Permission is hereby granted, free of charge, to any person obtaining a copy
 of this software and associated documentation files (the "Software"), to deal
 in the Software without restriction, including without limitation the rights
@@ -652,1010 +648,3 @@ hadoop-tools/hadoop-sls/src/main/html/js/thirdparty/d3.v3.js
 
 D3 is available under a 3-clause BSD license. For details, see:
 hadoop-tools/hadoop-sls/src/main/html/js/thirdparty/d3-LICENSE
-
-The binary distribution of this product bundles these dependencies under the
-following license:
-HSQLDB Database 2.0.0
---------------------------------------------------------------------------------
-"COPYRIGHTS AND LICENSES (based on BSD License)
-
-For work developed by the HSQL Development Group:
-
-Copyright (c) 2001-2016, The HSQL Development Group
-All rights reserved.
-
-Redistribution and use in source and binary forms, with or without
-modification, are permitted provided that the following conditions are met:
-
-Redistributions of source code must retain the above copyright notice, this
-list of conditions and the following disclaimer.
-
-Redistributions in binary form must reproduce the above copyright notice,
-this list of conditions and the following disclaimer in the documentation
-and/or other materials provided with the distribution.
-
-Neither the name of the HSQL Development Group nor the names of its
-contributors may be used to endorse or promote products derived from this
-software without specific prior written permission.
-
-THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS ""AS IS""
-AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE
-IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE
-ARE DISCLAIMED. IN NO EVENT SHALL HSQL DEVELOPMENT GROUP, HSQLDB.ORG,
-OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL,
-EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO,
-PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES;
-LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND
-ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
-(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS
-SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
-
-
-For work originally developed by the Hypersonic SQL Group:
-
-Copyright (c) 1995-2000 by the Hypersonic SQL Group.
-All rights reserved.
-Redistribution and use in source and binary forms, with or without
-modification, are permitted provided that the following conditions are met:
-
-Redistributions of source code must retain the above copyright notice, this
-list of conditions and the following disclaimer.
-
-Redistributions in binary form must reproduce the above copyright notice,
-this list of conditions and the following disclaimer in the documentation
-and/or other materials provided with the distribution.
-
-Neither the name of the Hypersonic SQL Group nor the names of its
-contributors may be used to endorse or promote products derived from this
-software without specific prior written permission.
-
-THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS ""AS IS""
-AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE
-IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE
-ARE DISCLAIMED. IN NO EVENT SHALL THE HYPERSONIC SQL GROUP,
-OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL,
-EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO,
-PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES;
-LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND
-ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
-(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS
-SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
-
-This software consists of voluntary contributions made by many individuals on behalf of the
-Hypersonic SQL Group."
-
-The binary distribution of this product bundles these dependencies under the
-following license:
-servlet-api 2.5
-jsp-api 2.1
-Streaming API for XML 1.0
---------------------------------------------------------------------------------
-COMMON DEVELOPMENT AND DISTRIBUTION LICENSE (CDDL) Version 1.0
-1.�Definitions.�
-
-1.1.�Contributor�means each individual or entity
-that creates or contributes to the creation of
-Modifications.�
-
-1.2.�Contributor Version�means the combination of the
-Original Software, prior Modifications used by a Contributor (if any), and the
-Modifications made by that particular Contributor.�
-
-1.3.�Covered
-Software�means (a) the Original Software, or (b) Modifications, or (c) the
-combination of files containing Original Software with files containing
-Modifications, in each case including portions
-thereof.�
-
-1.4.�Executable�means the Covered Software in any form other
-than Source Code.�
-
-1.5.�Initial Developer�means the individual or entity
-that first makes Original Software available under this
-License.�
-
-1.6.�Larger Work�means a work which combines Covered Software or
-portions thereof with code not governed by the terms of this
-License.�
-
-1.7.�License�means this document.�
-
-1.8.�Licensable�means
-having the right to grant, to the maximum extent possible, whether at the time
-of the initial grant or subsequently acquired, any and all of the rights
-conveyed herein.�
-
-1.9.�Modifications�means the Source Code and Executable
-form of any of the following:
-A. Any file that results from an addition to,
-deletion from or modification of the contents of a file containing Original
-Software or previous Modifications;
-B. Any new file that contains any part of the Original Software
-or previous Modification; or
-C. Any new file that is contributed or otherwise made available
-under the terms of this License.�
-
-1.10.�Original Software�means the Source Code and Executable form of
-computer software code that is originally released under this License.�
-
-1.11.�Patent Claims�means any patent claim(s), now owned or
-hereafter acquired, including without limitation, method, process, and apparatus
-claims, in any patent Licensable by grantor.�
-
-1.12.�Source Code�means (a) the common form of computer software code in which
-modifications are made and (b) associated documentation included in or
-with such code.�
-
-1.13.�You (or Your)�means an individual or a legal entity exercising rights
-under, and complying with all of the terms of, this License. For legal entities,
-You includes any entity which controls, is controlled by, or is under common control
-with You. For purposes of this definition, control means (a)�the power, direct
-or indirect, to cause the direction or management of such entity, whether by
-contract or otherwise, or (b)�ownership of more than fifty percent (50%) of the
-outstanding shares or beneficial ownership of such entity.�
-
-2. License Grants.
-
-2.1. The Initial Developer Grant. Conditioned upon Your compliance
-with Section 3.1 below and subject to third party intellectual property claims,
-the Initial Developer hereby grants You a world-wide, royalty-free,
-non-exclusive license:�
-
-(a) under intellectual property rights (other than
-patent or trademark) Licensable by Initial Developer, to use, reproduce, modify,
-display, perform, sublicense and distribute the Original Software (or portions
-thereof), with or without Modifications, and/or as part of a Larger Work;
-and�
-
-(b) under Patent Claims infringed by the making, using or selling of
-Original Software, to make, have made, use, practice, sell, and offer for sale,
-and/or otherwise dispose of the Original Software (or portions
-thereof);
-
-(c) The licenses granted in Sections�2.1(a) and (b) are
-effective on the date Initial Developer first distributes or otherwise makes the
-Original Software available to a third party under the terms of this
-License;
-
-(d) Notwithstanding Section�2.1(b) above, no patent license is
-granted: (1)�for code that You delete from the Original Software, or (2)�for
-infringements caused by: (i)�the modification of the Original Software, or
-(ii)�the combination of the Original Software with other software or
-devices.�
-
-2.2. Contributor Grant. Conditioned upon Your compliance with
-Section 3.1 below and subject to third party intellectual property claims, each
-Contributor hereby grants You a world-wide, royalty-free, non-exclusive
-license:�
-
-(a) under intellectual property rights (other than patent or
-trademark) Licensable by Contributor to use, reproduce, modify, display,
-perform, sublicense and distribute the Modifications created by such Contributor
-(or portions thereof), either on an unmodified basis, with other Modifications,
-as Covered Software and/or as part of a Larger Work; and�
-
-(b) under Patent
-Claims infringed by the making, using, or selling of Modifications made by that
-Contributor either alone and/or in combination with its Contributor Version (or
-portions of such combination), to make, use, sell, offer for sale, have made,
-and/or otherwise dispose of: (1)�Modifications made by that Contributor (or
-portions thereof); and (2)�the combination of Modifications made by that
-Contributor with its Contributor Version (or portions of such
-combination).�
-
-(c) The licenses granted in Sections�2.2(a) and 2.2(b) are
-effective on the date Contributor first distributes or otherwise makes the
-Modifications available to a third party.
-
-(d) Notwithstanding Section�2.2(b)
-above, no patent license is granted: (1)�for any code that Contributor has
-deleted from the Contributor Version; (2)�for infringements caused by:
-(i)�third party modifications of Contributor Version, or (ii)�the combination
-of Modifications made by that Contributor with other software (except as part of
-the Contributor Version) or other devices; or (3)�under Patent Claims infringed
-by Covered Software in the absence of Modifications made by that
-Contributor.�
-
-3. Distribution Obligations.�
-
-3.1. Availability of Source
-Code. Any Covered Software that You distribute or otherwise make available in
-Executable form must also be made available in Source Code form and that Source
-Code form must be distributed only under the terms of this License. You must
-include a copy of this License with every copy of the Source Code form of the
-Covered Software You distribute or otherwise make available. You must inform
-recipients of any such Covered Software in Executable form as to how they can
-obtain such Covered Software in Source Code form in a reasonable manner on or
-through a medium customarily used for software exchange.�
-
-3.2.
-Modifications. The Modifications that You create or to which You contribute are
-governed by the terms of this License. You represent that You believe Your
-Modifications are Your original creation(s) and/or You have sufficient rights to
-grant the rights conveyed by this License.�
-
-3.3. Required Notices. You must
-include a notice in each of Your Modifications that identifies You as the
-Contributor of the Modification. You may not remove or alter any copyright,
-patent or trademark notices contained within the Covered Software, or any
-notices of licensing or any descriptive text giving attribution to any
-Contributor or the Initial Developer.�
-
-3.4. Application of Additional Terms.
-You may not offer or impose any terms on any Covered Software in Source Code
-form that alters or restricts the applicable version of this License or the
-recipients rights hereunder. You may choose to offer, and to charge a fee for,
-warranty, support, indemnity or liability obligations to one or more recipients
-of Covered Software. However, you may do so only on Your own behalf, and not on
-behalf of the Initial Developer or any Contributor. You must make it absolutely
-clear that any such warranty, support, indemnity or liability obligation is
-offered by You alone, and You hereby agree to indemnify the Initial Developer
-and every Contributor for any liability incurred by the Initial Developer or
-such Contributor as a result of warranty, support, indemnity or liability terms
-You offer.
-
-3.5. Distribution of Executable Versions. You may distribute the
-Executable form of the Covered Software under the terms of this License or under
-the terms of a license of Your choice, which may contain terms different from
-this License, provided that You are in compliance with the terms of this License
-and that the license for the Executable form does not attempt to limit or alter
-the recipients rights in the Source Code form from the rights set forth in this
-License. If You distribute the Covered Software in Executable form under a
-different license, You must make it absolutely clear that any terms which differ
-from this License are offered by You alone, not by the Initial Developer or
-Contributor. You hereby agree to indemnify the Initial Developer and every
-Contributor for any liability incurred by the Initial Developer or such
-Contributor as a result of any such terms You offer.�
-
-3.6. Larger Works. You
-may create a Larger Work by combining Covered Software with other code not
-governed by the terms of this License and distribute the Larger Work as a single
-product. In such a case, You must make sure the requirements of this License are
-fulfilled for the Covered Software.�
-
-4. Versions of the License.�
-
-4.1.
-New Versions. Sun Microsystems, Inc. is the initial license steward and may
-publish revised and/or new versions of this License from time to time. Each
-version will be given a distinguishing version number. Except as provided in
-Section 4.3, no one other than the license steward has the right to modify this
-License.�
-
-4.2. Effect of New Versions. You may always continue to use,
-distribute or otherwise make the Covered Software available under the terms of
-the version of the License under which You originally received the Covered
-Software. If the Initial Developer includes a notice in the Original Software
-prohibiting it from being distributed or otherwise made available under any
-subsequent version of the License, You must distribute and make the Covered
-Software available under the terms of the version of the License under which You
-originally received the Covered Software. Otherwise, You may also choose to use,
-distribute or otherwise make the Covered Software available under the terms of
-any subsequent version of the License published by the license
-steward.�
-
-4.3. Modified Versions. When You are an Initial Developer and You
-want to create a new license for Your Original Software, You may create and use
-a modified version of this License if You: (a)�rename the license and remove
-any references to the name of the license steward (except to note that the
-license differs from this License); and (b)�otherwise make it clear that the
-license contains terms which differ from this License.�
-
-5. DISCLAIMER OF WARRANTY.
-
-COVERED SOFTWARE IS PROVIDED UNDER THIS LICENSE ON AN AS IS BASIS,
-WITHOUT WARRANTY OF ANY KIND, EITHER EXPRESSED OR IMPLIED, INCLUDING, WITHOUT
-LIMITATION, WARRANTIES THAT THE COVERED SOFTWARE IS FREE OF DEFECTS,
-MERCHANTABLE, FIT FOR A PARTICULAR PURPOSE OR NON-INFRINGING. THE ENTIRE RISK AS
-TO THE QUALITY AND PERFORMANCE OF THE COVERED SOFTWARE IS WITH YOU. SHOULD ANY
-COVERED SOFTWARE PROVE DEFECTIVE IN ANY RESPECT, YOU (NOT THE INITIAL DEVELOPER
-OR ANY OTHER CONTRIBUTOR) ASSUME THE COST OF ANY NECESSARY SERVICING, REPAIR OR
-CORRECTION. THIS DISCLAIMER OF WARRANTY CONSTITUTES AN ESSENTIAL PART OF THIS
-LICENSE. NO USE OF ANY COVERED SOFTWARE IS AUTHORIZED HEREUNDER EXCEPT UNDER
-THIS DISCLAIMER.�
-
-6. TERMINATION.�
-
-6.1. This License and the rights
-granted hereunder will terminate automatically if You fail to comply with terms
-herein and fail to cure such breach within 30 days of becoming aware of the
-breach. Provisions which, by their nature, must remain in effect beyond the
-termination of this License shall survive.�
-
-6.2. If You assert a patent
-infringement claim (excluding declaratory judgment actions) against Initial
-Developer or a Contributor (the Initial Developer or Contributor against whom
-You assert such claim is referred to as Participant) alleging that the
-Participant Software (meaning the Contributor Version where the Participant is a
-Contributor or the Original Software where the Participant is the Initial
-Developer) directly or indirectly infringes any patent, then any and all rights
-granted directly or indirectly to You by such Participant, the Initial Developer
-(if the Initial Developer is not the Participant) and all Contributors under
-Sections�2.1 and/or 2.2 of this License shall, upon 60 days notice from
-Participant terminate prospectively and automatically at the expiration of such
-60 day notice period, unless if within such 60 day period You withdraw Your
-claim with respect to the Participant Software against such Participant either
-unilaterally or pursuant to a written agreement with Participant.�
-
-6.3. In
-the event of termination under Sections�6.1 or 6.2 above, all end user licenses
-that have been validly granted by You or any distributor hereunder prior to
-termination (excluding licenses granted to You by any distributor) shall survive
-termination.�
-
-7. LIMITATION OF LIABILITY.
-UNDER NO CIRCUMSTANCES AND UNDER
-NO LEGAL THEORY, WHETHER TORT (INCLUDING NEGLIGENCE), CONTRACT, OR OTHERWISE,
-SHALL YOU, THE INITIAL DEVELOPER, ANY OTHER CONTRIBUTOR, OR ANY DISTRIBUTOR OF
-COVERED SOFTWARE, OR ANY SUPPLIER OF ANY OF SUCH PARTIES, BE LIABLE TO ANY
-PERSON FOR ANY INDIRECT, SPECIAL, INCIDENTAL, OR CONSEQUENTIAL DAMAGES OF ANY
-CHARACTER INCLUDING, WITHOUT LIMITATION, DAMAGES FOR LOST PROFITS, LOSS OF
-GOODWILL, WORK STOPPAGE, COMPUTER FAILURE OR MALFUNCTION, OR ANY AND ALL OTHER
-COMMERCIAL DAMAGES OR LOSSES, EVEN IF SUCH PARTY SHALL HAVE BEEN INFORMED OF THE
-POSSIBILITY OF SUCH DAMAGES. THIS LIMITATION OF LIABILITY SHALL NOT APPLY TO
-LIABILITY FOR DEATH OR PERSONAL INJURY RESULTING FROM SUCH PARTYS NEGLIGENCE TO
-THE EXTENT APPLICABLE LAW PROHIBITS SUCH LIMITATION. SOME JURISDICTIONS DO NOT
-ALLOW THE EXCLUSION OR LIMITATION OF INCIDENTAL OR CONSEQUENTIAL DAMAGES, SO
-THIS EXCLUSION AND LIMITATION MAY NOT APPLY TO YOU.�
-
-8. U.S. GOVERNMENT END USERS.
-
-The Covered Software is a commercial item, as that term is defined in
-48�C.F.R.�2.101 (Oct. 1995), consisting of commercial computer software (as
-that term is defined at 48 C.F.R. �252.227-7014(a)(1)) and commercial computer
-software documentation as such terms are used in 48�C.F.R.�12.212 (Sept.
-1995). Consistent with 48 C.F.R. 12.212 and 48 C.F.R. 227.7202-1 through
-227.7202-4 (June 1995), all U.S. Government End Users acquire Covered Software
-with only those rights set forth herein. This U.S. Government Rights clause is
-in lieu of, and supersedes, any other FAR, DFAR, or other clause or provision
-that addresses Government rights in computer software under this
-License.�
-
-9. MISCELLANEOUS.
-This License represents the complete agreement
-concerning subject matter hereof. If any provision of this License is held to be
-unenforceable, such provision shall be reformed only to the extent necessary to
-make it enforceable. This License shall be governed by the law of the
-jurisdiction specified in a notice contained within the Original Software
-(except to the extent applicable law, if any, provides otherwise), excluding
-such jurisdictions conflict-of-law provisions. Any litigation relating to this
-License shall be subject to the jurisdiction of the courts located in the
-jurisdiction and venue specified in a notice contained within the Original
-Software, with the losing party responsible for costs, including, without
-limitation, court costs and reasonable attorneys fees and expenses. The
-application of the United Nations Convention on Contracts for the International
-Sale of Goods is expressly excluded. Any law or regulation which provides that
-the language of a contract shall be construed against the drafter shall not
-apply to this License. You agree that You alone are responsible for compliance
-with the United States export administration regulations (and the export control
-laws and regulation of any other countries) when You use, distribute or
-otherwise make available any Covered Software.�
-
-10. RESPONSIBILITY FOR CLAIMS.
-As between Initial Developer and the Contributors, each party is
-responsible for claims and damages arising, directly or indirectly, out of its
-utilization of rights under this License and You agree to work with Initial
-Developer and Contributors to distribute such responsibility on an equitable
-basis. Nothing herein is intended or shall be deemed to constitute any admission
-of liability.�
-
-The binary distribution of this product bundles these dependencies under the
-following license:
-Jersey 1.9
-JAXB API bundle for GlassFish V3 2.2.2
-JAXB RI 2.2.3
---------------------------------------------------------------------------------
-COMMON DEVELOPMENT AND DISTRIBUTION LICENSE (CDDL)Version 1.1
-
-1. Definitions.
-
-1.1. \u201cContributor\u201d means each individual or entity that creates or
-contributes to the creation of Modifications.
-1.2. \u201cContributor Version\u201d means the combination of the Original Software,
-prior Modifications used by a Contributor (if any), and the Modifications made
-by that particular Contributor.
-1.3. \u201cCovered Software\u201d means (a) the Original Software, or (b)
-Modifications, or (c) the combination of files containing Original Software with
-files containing Modifications, in each case including portions thereof.
-1.4. \u201cExecutable\u201d means the Covered Software in any form other than Source
-Code.
-1.5. \u201cInitial Developer\u201d means the individual or entity that first makes
-Original Software available under this License.
-1.6. \u201cLarger Work\u201d means a work which combines Covered Software or portions
-thereof with code not governed by the terms of this License.
-1.7. \u201cLicense\u201d means this document.
-1.8. \u201cLicensable\u201d means having the right to grant, to the maximum extent
-possible, whether at the time of the initial grant or subsequently acquired, any
-and all of the rights conveyed herein.
-1.9. \u201cModifications\u201d means the Source Code and Executable form of any of the
-following:
-A. Any file that results from an addition to, deletion from or modification of
-the contents of a file containing Original Software or previous Modifications;
-B. Any new file that contains any part of the Original Software or previous
-Modification; or
-C. Any new file that is contributed or otherwise made available under the terms
-of this License.
-1.10. \u201cOriginal Software\u201d means the Source Code and Executable form of
-computer software code that is originally released under this License.
-1.11. \u201cPatent Claims\u201d means any patent claim(s), now owned or hereafter
-acquired, including without limitation, method, process, and apparatus claims,
-in any patent Licensable by grantor.
-1.12. \u201cSource Code\u201d means (a) the common form of computer software code in
-which modifications are made and (b) associated documentation included in or
-with such code.
-1.13. \u201cYou\u201d (or \u201cYour\u201d) means an individual or a legal entity exercising
-rights under, and complying with all of the terms of, this License. For legal
-entities, \u201cYou\u201d includes any entity which controls, is controlled by, or is
-under common control with You. For purposes of this definition, \u201ccontrol\u201d
-means (a) the power, direct or indirect, to cause the direction or management of
-such entity, whether by contract or otherwise, or (b) ownership of more than
-fifty percent (50%) of the outstanding shares or beneficial ownership of such
-entity.
-
-2. License Grants.
-
-2.1. The Initial Developer Grant.
-
-Conditioned upon Your compliance with Section 3.1 below and subject to
-third party intellectual property claims, the Initial Developer hereby grants
-You a world-wide, royalty-free, non-exclusive license:
-(a) under intellectual
-property rights (other than patent or trademark) Licensable by Initial
-Developer, to use, reproduce, modify, display, perform, sublicense and
-distribute the Original Software (or portions thereof), with or without
-Modifications, and/or as part of a Larger Work; and
-(b) under Patent Claims
-infringed by the making, using or selling of Original Software, to make, have
-made, use, practice, sell, and offer for sale, and/or otherwise dispose of the
-Original Software (or portions thereof).
-(c) The licenses granted in Sections
-2.1(a) and (b) are effective on the date Initial Developer first distributes or
-otherwise makes the Original Software available to a third party under the terms
-of this License.
-(d) Notwithstanding Section 2.1(b) above, no patent license is
-granted: (1) for code that You delete from the Original Software, or (2) for
-infringements caused by: (i) the modification of the Original Software, or (ii)
-the combination of the Original Software with other software or devices.
-
-2.2. Contributor Grant.
-
-Conditioned upon Your compliance with Section 3.1 below and
-subject to third party intellectual property claims, each Contributor hereby
-grants You a world-wide, royalty-free, non-exclusive license:
-(a) under
-intellectual property rights (other than patent or trademark) Licensable by
-Contributor to use, reproduce, modify, display, perform, sublicense and
-distribute the Modifications created by such Contributor (or portions thereof),
-either on an unmodified basis, with other Modifications, as Covered Software
-and/or as part of a Larger Work; and
-(b) under Patent Claims infringed by the
-making, using, or selling of Modifications made by that Contributor either alone
-and/or in combination with its Contributor Version (or portions of such
-combination), to make, use, sell, offer for sale, have made, and/or otherwise
-dispose of: (1) Modifications made by that Contributor (or portions thereof);
-and (2) the combination of Modifications made by that Contributor with its
-Contributor Version (or portions of such combination).
-(c) The licenses granted
-in Sections 2.2(a) and 2.2(b) are effective on the date Contributor first
-distributes or otherwise makes the Modifications available to a third
-party.
-(d) Notwithstanding Section 2.2(b) above, no patent license is granted:
-(1) for any code that Contributor has deleted from the Contributor Version; (2)
-for infringements caused by: (i) third party modifications of Contributor
-Version, or (ii) the combination of Modifications made by that Contributor with
-other software (except as part of the Contributor Version) or other devices; or
-(3) under Patent Claims infringed by Covered Software in the absence of
-Modifications made by that Contributor.
-
-3. Distribution Obligations.
-
-3.1. Availability of Source Code.
-Any Covered Software that You distribute or
-otherwise make available in Executable form must also be made available in
-Source Code form and that Source Code form must be distributed only under the
-terms of this License. You must include a copy of this License with every copy
-of the Source Code form of the Covered Software You distribute or otherwise make
-available. You must inform recipients of any such Covered Software in Executable
-form as to how they can obtain such Covered Software in Source Code form in a
-reasonable manner on or through a medium customarily used for software
-exchange.
-3.2. Modifications.
-The Modifications that You create or to which
-You contribute are governed by the terms of this License. You represent that You
-believe Your Modifications are Your original creation(s) and/or You have
-sufficient rights to grant the rights conveyed by this License.
-3.3. Required Notices.
-You must include a notice in each of Your Modifications that
-identifies You as the Contributor of the Modification. You may not remove or
-alter any copyright, patent or trademark notices contained within the Covered
-Software, or any notices of licensing or any descriptive text giving attribution
-to any Contributor or the Initial Developer.
-3.4. Application of Additional Terms.
-You may not offer or impose any terms on any Covered Software in Source
-Code form that alters or restricts the applicable version of this License or the
-recipients' rights hereunder. You may choose to offer, and to charge a fee for,
-warranty, support, indemnity or liability obligations to one or more recipients
-of Covered Software. However, you may do so only on Your own behalf, and not on
-behalf of the Initial Developer or any Contributor. You must make it absolutely
-clear that any such warranty, support, indemnity or liability obligation is
-offered by You alone, and You hereby agree to indemnify the Initial Developer
-and every Contributor for any liability incurred by the Initial Developer or
-such Contributor as a result of warranty, support, indemnity or liability terms
-You offer.
-3.5. Distribution of Executable Versions.
-You may distribute the
-Executable form of the Covered Software under the terms of this License or under
-the terms of a license of Your choice, which may contain terms different from
-this License, provided that You are in compliance with the terms of this License
-and that the license for the Executable form does not attempt to limit or alter
-the recipient's rights in the Source Code form from the rights set forth in
-this License. If You distribute the Covered Software in Executable form under a
-different license, You must make it absolutely clear that any terms which differ
-from this License are offered by You alone, not by the Initial Developer or
-Contributor. You hereby agree to indemnify the Initial Developer and every
-Contributor for any liability incurred by the Initial Developer or such
-Contributor as a result of any such terms You offer.
-3.6. Larger Works.
-You
-may create a Larger Work by combining Covered Software with other code not
-governed by the terms of this License and distribute the Larger Work as a single
-product. In such a case, You must make sure the requirements of this License are
-fulfilled for the Covered Software.
-
-4. Versions of the License.
-
-4.1. New Versions.
-Oracle is the initial license steward and may publish revised and/or
-new versions of this License from time to time. Each version will be given a
-distinguishing version number. Except as provided in Section 4.3, no one other
-than the license steward has the right to modify this License.
-4.2. Effect of New Versions.
-You may always continue to use, distribute or otherwise make the
-Covered Software available under the terms of the version of the License under
-which You originally received the Covered Software. If the Initial Developer
-includes a notice in the Original Software prohibiting it from being distributed
-or otherwise made available under any subsequent version of the License, You
-must distribute and make the Covered Software available under the terms of the
-version of the License under which You originally received the Covered Software.
-Otherwise, You may also choose to use, distribute or otherwise make the Covered
-Software available under the terms of any subsequent version of the License
-published by the license steward.
-4.3. Modified Versions.
-When You are an
-Initial Developer and You want to create a new license for Your Original
-Software, You may create and use a modified version of this License if You: (a)
-rename the license and remove any references to the name of the license steward
-(except to note that the license differs from this License); and (b) otherwise
-make it clear that the license contains terms which differ from this
-License.
-
-5. DISCLAIMER OF WARRANTY.
-
-COVERED SOFTWARE IS PROVIDED UNDER THIS
-LICENSE ON AN \u201cAS IS\u201d BASIS, WITHOUT WARRANTY OF ANY KIND, EITHER EXPRESSED
-OR IMPLIED, INCLUDING, WITHOUT LIMITATION, WARRANTIES THAT THE COVERED SOFTWARE
-IS FREE OF DEFECTS, MERCHANTABLE, FIT FOR A PARTICULAR PURPOSE OR
-NON-INFRINGING. THE ENTIRE RISK AS TO THE QUALITY AND PERFORMANCE OF THE COVERED
-SOFTWARE IS WITH YOU. SHOULD ANY COVERED SOFTWARE PROVE DEFECTIVE IN ANY
-RESPECT, YOU (NOT THE INITIAL DEVELOPER OR ANY OTHER CONTRIBUTOR) ASSUME THE
-COST OF ANY NECESSARY SERVICING, REPAIR OR CORRECTION. THIS DISCLAIMER OF
-WARRANTY CONSTITUTES AN ESSENTIAL PART OF THIS LICENSE. NO USE OF ANY COVERED
-SOFTWARE IS AUTHORIZED HEREUNDER EXCEPT UNDER THIS DISCLAIMER.
-
-6. TERMINATION.
-
-6.1. This License and the rights granted hereunder will
-terminate automatically if You fail to comply with terms herein and fail to cure
-such breach within 30 days of becoming aware of the breach. Provisions which, by
-their nature, must remain in effect beyond the termination of this License shall
-survive.
-6.2. If You assert a patent infringement claim (excluding declaratory
-judgment actions) against Initial Developer or a Contributor (the Initial
-Developer or Contributor against whom You assert such claim is referred to as
-\u201cParticipant\u201d) alleging that the Participant Software (meaning the
-Contributor Version where the Participant is a Contributor or the Original
-Software where the Participant is the Initial Developer) directly or indirectly
-infringes any patent, then any and all rights granted directly or indirectly to
-You by such Participant, the Initial Developer (if the Initial Developer is not
-the Participant) and all Contributors under Sections 2.1 and/or 2.2 of this
-License shall, upon 60 days notice from Participant terminate prospectively and
-automatically at the expiration of such 60 day notice period, unless if within
-such 60 day period You withdraw Your claim with respect to the Participant
-Software against such Participant either unilaterally or pursuant to a written
-agreement with Participant.
-6.3. If You assert a patent infringement claim
-against Participant alleging that the Participant Software directly or
-indirectly infringes any patent where such claim is resolved (such as by license
-or settlement) prior to the initiation of patent infringement litigation, then
-the reasonable value of the licenses granted by such Participant under Sections
-2.1 or 2.2 shall be taken into account in determining the amount or value of any
-payment or license.
-6.4. In the event of termination under Sections 6.1 or 6.2
-above, all end user licenses that have been validly granted by You or any
-distributor hereunder prior to termination (excluding licenses granted to You by
-any distributor) shall survive termination.
-
-7. LIMITATION OF LIABILITY.
-
-UNDER NO CIRCUMSTANCES AND UNDER NO LEGAL THEORY, WHETHER TORT
-(INCLUDING NEGLIGENCE), CONTRACT, OR OTHERWISE, SHALL YOU, THE INITIAL
-DEVELOPER, ANY OTHER CONTRIBUTOR, OR ANY DISTRIBUTOR OF COVERED SOFTWARE, OR ANY
-SUPPLIER OF ANY OF SUCH PARTIES, BE LIABLE TO ANY PERSON FOR ANY INDIRECT,
-SPECIAL, INCIDENTAL, OR CONSEQUENTIAL DAMAGES OF ANY CHARACTER INCLUDING,
-WITHOUT LIMITATION, DAMAGES FOR LOSS OF GOODWILL, WORK STOPPAGE, COMPUTER
-FAILURE OR MALFUNCTION, OR ANY AND ALL OTHER COMMERCIAL DAMAGES OR LOSSES, EVEN
-IF SUCH PARTY SHALL HAVE BEEN INFORMED OF THE POSSIBILITY OF SUCH DAMAGES. THIS
-LIMITATION OF LIABILITY SHALL NOT APPLY TO LIABILITY FOR DEATH OR PERSONAL
-INJURY RESULTING FROM SUCH PARTY'S NEGLIGENCE TO THE EXTENT APPLICABLE LAW
-PROHIBITS SUCH LIMITATION. SOME JURISDICTIONS DO NOT ALLOW THE EXCLUSION OR
-LIMITATION OF INCIDENTAL OR CONSEQUENTIAL DAMAGES, SO THIS EXCLUSION AND
-LIMITATION MAY NOT APPLY TO YOU.
-
-8. U.S. GOVERNMENT END USERS.
-
-The Covered
-Software is a \u201ccommercial item,\u201d as that term is defined in 48 C.F.R. 2.101
-(Oct. 1995), consisting of \u201ccommercial computer software\u201d (as that term is
-defined at 48 C.F.R. � 252.227-7014(a)(1)) and \u201ccommercial computer software
-documentation\u201d as such terms are used in 48 C.F.R. 12.212 (Sept. 1995).
-Consistent with 48 C.F.R. 12.212 and 48 C.F.R. 227.7202-1 through 227.7202-4
-(June 1995), all U.S. Government End Users acquire Covered Software with only
-those rights set forth herein. This U.S. Government Rights clause is in lieu of,
-and supersedes, any other FAR, DFAR, or other clause or provision that addresses
-Government rights in computer software under this License.
-
-9. MISCELLANEOUS.
-
-This License represents the complete agreement concerning
-subject matter hereof. If any provision of this License is held to be
-unenforceable, such provision shall be reformed only to the extent necessary to
-make it enforceable. This License shall be governed by the law of the
-jurisdiction specified in a notice contained within the Original Software
-(except to the extent applicable law, if any, provides otherwise), excluding
-such jurisdiction's conflict-of-law provisions. Any litigation relating to this
-License shall be subject to the jurisdiction of the courts located in the
-jurisdiction and venue specified in a notice contained within the Original
-Software, with the losing party responsible for costs, including, without
-limitation, court costs and reasonable attorneys' fees and expenses. The
-application of the United Nations Convention on Contracts for the International
-Sale of Goods is expressly excluded. Any law or regulation which provides that
-the language of a contract shall be construed against the drafter shall not
-apply to this License. You agree that You alone are responsible for compliance
-with the United States export administration regulations (and the export control
-laws and regulation of any other countries) when You use, distribute or
-otherwise make available any Covered Software.
-
-10. RESPONSIBILITY FOR CLAIMS.
-
-As between Initial Developer and the Contributors, each party is
-responsible for claims and damages arising, directly or indirectly, out of its
-utilization of rights under this License and You agree to work with Initial
-Developer and Contributors to distribute such responsibility on an equitable
-basis. Nothing herein is intended or shall be deemed to constitute any admission
-of liability.
-
-The binary distribution of this product bundles these dependencies under the
-following license:
-Protocol Buffer Java API 2.5.0
---------------------------------------------------------------------------------
-This license applies to all parts of Protocol Buffers except the following:
-
-  - Atomicops support for generic gcc, located in
-    src/google/protobuf/stubs/atomicops_internals_generic_gcc.h.
-    This file is copyrighted by Red Hat Inc.
-
-  - Atomicops support for AIX/POWER, located in
-    src/google/protobuf/stubs/atomicops_internals_power.h.
-    This file is copyrighted by Bloomberg Finance LP.
-
-Copyright 2014, Google Inc.  All rights reserved.
-
-Redistribution and use in source and binary forms, with or without
-modification, are permitted provided that the following conditions are
-met:
-
-    * Redistributions of source code must retain the above copyright
-notice, this list of conditions and the following disclaimer.
-    * Redistributions in binary form must reproduce the above
-copyright notice, this list of conditions and the following disclaimer
-in the documentation and/or other materials provided with the
-distribution.
-    * Neither the name of Google Inc. nor the names of its
-contributors may be used to endorse or promote products derived from
-this software without specific prior written permission.
-
-THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
-"AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
-LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
-A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
-OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
-SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
-LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
-DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
-THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
-(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
-OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
-
-Code generated by the Protocol Buffer compiler is owned by the owner
-of the input file used when generating it.  This code is not
-standalone and requires a support library to be linked with it.  This
-support library is itself covered by the above license.
-
-For:
-XML Commons External Components XML APIs 1.3.04
---------------------------------------------------------------------------------
-By obtaining, using and/or copying this work, you (the licensee) agree that you
-have read, understood, and will comply with the following terms and conditions.
-
-Permission to copy, modify, and distribute this software and its documentation,
-with or without modification, for any purpose and without fee or royalty is
-hereby granted, provided that you include the following on ALL copies of the
-software and documentation or portions thereof, including modifications:
-- The full text of this NOTICE in a location viewable to users of the
-redistributed or derivative work.
-- Any pre-existing intellectual property disclaimers, notices, or terms and
-conditions. If none exist, the W3C Software Short Notice should be included
-(hypertext is preferred, text is permitted) within the body of any redistributed
-or derivative code.
-- Notice of any changes or modifications to the files, including the date changes
-were made. (We recommend you provide URIs to the location from which the code is
-derived.)
-
-The binary distribution of this product bundles these dependencies under the
-following license:
-JUnit 4.11
-ecj-4.3.1.jar
---------------------------------------------------------------------------------
-Eclipse Public License - v 1.0
-
-THE ACCOMPANYING PROGRAM IS PROVIDED UNDER THE TERMS OF THIS ECLIPSE PUBLIC
-LICENSE ("AGREEMENT"). ANY USE, REPRODUCTION OR DISTRIBUTION OF THE PROGRAM
-CONSTITUTES RECIPIENT'S ACCEPTANCE OF THIS AGREEMENT.
-
-1. DEFINITIONS
-
-"Contribution" means:
-
-a) in the case of the initial Contributor, the initial code and documentation
-distributed under this Agreement, and
-b) in the case of each subsequent Contributor:
-i) changes to the Program, and
-ii) additions to the Program;
-where such changes and/or additions to the Program originate from and are
-distributed by that particular Contributor. A Contribution 'originates' from a
-Contributor if it was added to the Program by such Contributor itself or anyone
-acting on such Contributor's behalf. Contributions do not include additions to
-the Program which: (i) are separate modules of software distributed in
-conjunction with the Program under their own license agreement, and (ii) are not
-derivative works of the Program.
-"Contributor" means any person or entity that distributes the Program.
-
-"Licensed Patents" mean patent claims licensable by a Contributor which are
-necessarily infringed by the use or sale of its Contribution alone or when
-combined with the Program.
-
-"Program" means the Contributions distributed in accordance with this Agreement.
-
-"Recipient" means anyone who receives the Program under this Agreement,
-including all Contributors.
-
-2. GRANT OF RIGHTS
-
-a) Subject to the terms of this Agreement, each Contributor hereby grants
-Recipient a non-exclusive, worldwide, royalty-free copyright license to
-reproduce, prepare derivative works of, publicly display, publicly perform,
-distribute and sublicense the Contribution of such Contributor, if any, and such
-derivative works, in source code and object code form.
-b) Subject to the terms of this Agreement, each Contributor hereby grants
-Recipient a non-exclusive, worldwide, royalty-free patent license under Licensed
-Patents to make, use, sell, offer to sell, import and otherwise transfer the
-Contribution of such Contributor, if any, in source code and object code form.
-This patent license shall apply to the combination of the Contribution and the
-Program if, at the time the Contribution is added by the Contributor, such
-addition of the Contribution causes such combination to be covered by the
-Licensed Patents. The patent license shall not apply to any other combinations
-which include the Contribution. No hardware per se is licensed hereunder.
-c) Recipient understands that although each Contributor grants the licenses to
-its Contributions set forth herein, no assurances are provided by any
-Contributor that the Program does not infringe the patent or other intellectual
-property rights of any other entity. Each Contributor disclaims any liability to
-Recipient for claims brought by any other entity based on infringement of
-intellectual property rights or otherwise. As a condition to exercising the
-rights and licenses granted hereunder, each Recipient hereby assumes sole
-responsibility to secure any other intellectual property rights needed, if any.
-For example, if a third party patent license is required to allow Recipient to
-distribute the Program, it is Recipient's responsibility to acquire that license
-before distributing the Program.
-d) Each Contributor represents that to its knowledge it has sufficient copyright
-rights in its Contribution, if any, to grant the copyright license set forth in
-this Agreement.
-3. REQUIREMENTS
-
-A Contributor may choose to distribute the Program in object code form under its
-own license agreement, provided that:
-
-a) it complies with the terms and conditions of this Agreement; and
-b) its license agreement:
-i) effectively disclaims on behalf of all Contributors all warranties and
-conditions, express and implied, including warranties or conditions of title and
-non-infringement, and implied warranties or conditions of merchantability and
-fitness for a particular purpose;
-ii) effectively excludes on behalf of all Contributors all liability for
-damages, including direct, indirect, special, incidental and consequential
-damages, such as lost profits;
-iii) states that any provisions which differ from this Agreement are offered by
-that Contributor alone and not by any other party; and
-iv) states that source code for the Program is available from such Contributor,
-and informs licensees how to obtain it in a reasonable manner on or through a
-medium customarily used for software exchange.
-When the Program is made available in source code form:
-
-a) it must be made available under this Agreement; and
-b) a copy of this Agreement must be included with each copy of the Program.
-Contributors may not remove or alter any copyright notices contained within the
-Program.
-
-Each Contributor must identify itself as the originator of its Contribution, if
-any, in a manner that reasonably allows subsequent Recipients to identify the
-originator of the Contribution.
-
-4. COMMERCIAL DISTRIBUTION
-
-Commercial distributors of software may accept certain responsibilities with
-respect to end users, business partners and the like. While this license is
-intended to facilitate the commercial use of the Program, the Contributor who
-includes the Program in a commercial product offering should do so in a manner
-which does not create potential liability for other Contributors. Therefore, if
-a Contributor includes the Program in a commercial product offering, such
-Contributor ("Commercial Contributor") hereby agrees to defend and indemnify
-every other Contributor ("Indemnified Contributor") against any losses, damages
-and costs (collectively "Losses") arising from claims, lawsuits and other legal
-actions brought by a third party against the Indemnified Contributor to the
-extent caused by the acts or omissions of such Commercial Contributor in
-connection with its distribution of the Program in a commercial product
-offering. The obligations in this section do not apply to any claims or Losses
-relating to any actual or alleged intellectual property infringement. In order
-to qualify, an Indemnified Contributor must: a) promptly notify the Commercial
-Contributor in writing of such claim, and b) allow the Commercial Contributor to
-control, and cooperate with the Commercial Contributor in, the defense and any
-related settlement negotiations. The Indemnified Contributor may participate in
-any such claim at its own expense.
-
-For example, a Contributor might include the Program in a commercial product
-offering, Product X. That Contributor is then a Commercial Contributor. If that
-Commercial Contributor then makes performance claims, or offers warranties
-related to Product X, those performance claims and warranties are such
-Commercial Contributor's responsibility alone. Under this section, the
-Commercial Contributor would have to defend claims against the other
-Contributors related to those performance claims and warranties, and if a court
-requires any other Contributor to pay any damages as a result, the Commercial
-Contributor must pay those damages.
-
-5. NO WARRANTY
-
-EXCEPT AS EXPRESSLY SET FORTH IN THIS AGREEMENT, THE PROGRAM IS PROVIDED ON AN
-"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, EITHER EXPRESS OR
-IMPLIED INCLUDING, WITHOUT LIMITATION, ANY WARRANTIES OR CONDITIONS OF TITLE,
-NON-INFRINGEMENT, MERCHANTABILITY OR FITNESS FOR A PARTICULAR PURPOSE. Each
-Recipient is solely responsible for determining the appropriateness of using and
-distributing the Program and assumes all risks associated with its exercise of
-rights under this Agreement , including but not limited to the risks and costs
-of program errors, compliance with applicable laws, damage to or loss of data,
-programs or equipment, and unavailability or interruption of operations.
-
-6. DISCLAIMER OF LIABILITY
-
-EXCEPT AS EXPRESSLY SET FORTH IN THIS AGREEMENT, NEITHER RECIPIENT NOR ANY
-CONTRIBUTORS SHALL HAVE ANY LIABILITY FOR ANY DIRECT, INDIRECT, INCIDENTAL,
-SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING WITHOUT LIMITATION LOST
-PROFITS), HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT,
-STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY
-OUT OF THE USE OR DISTRIBUTION OF THE PROGRAM OR THE EXERCISE OF ANY RIGHTS
-GRANTED HEREUNDER, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGES.
-
-7. GENERAL
-
-If any provision of this Agreement is invalid or unenforceable under applicable
-law, it shall not affect the validity or enforceability of the remainder of the
-terms of this Agreement, and without further action by the parties hereto, such
-provision shall be reformed to the minimum extent necessary to make such
-provision valid and enforceable.
-
-If Recipient institutes patent litigation against any entity (including a
-cross-claim or counterclaim in a lawsuit) alleging that the Program itself
-(excluding combinations of the Program with other software or hardware)
-infringes such Recipient's patent(s), then such Recipient's rights granted under
-Section 2(b) shall terminate as of the date such litigation is filed.
-
-All Recipient's rights under this Agreement shall terminate if it fails to
-comply with any of the material terms or conditions of this Agreement and does
-not cure such failure in a reasonable period of time after becoming aware of
-such noncompliance. If all Recipient's rights under this Agreement terminate,
-Recipient agrees to cease use and distribution of the Program as soon as
-reasonably practicable. However, Recipient's obligations under this Agreement
-and any licenses granted by Recipient relating to the Program shall continue and
-survive.
-
-Everyone is permitted to copy and distribute copies of this Agreement, but in
-order to avoid inconsistency the Agreement is copyrighted and may only be
-modified in the following manner. The Agreement Steward reserves the right to
-publish new versions (including revisions) of this Agreement from time to time.
-No one other than the Agreement Steward has the right to modify this Agreement.
-The Eclipse Foundation is the initial Agreement Steward. The Eclipse Foundation
-may assign the responsibility to serve as the Agreement Steward to a suitable
-separate entity. Each new version of the Agreement will be given a
-distinguishing version number. The Program (including Contributions) may always
-be distributed subject to the version of the Agreement under which it was
-received. In addition, after a new version of the Agreement is published,
-Contributor may elect to distribute the Program (including its Contributions)
-under the new version. Except as expressly stated in Sections 2(a) and 2(b)
-above, Recipient receives no rights or licenses to the intellectual property of
-any Contributor under this Agreement, whether expressly, by implication,
-estoppel or otherwise. All rights in the Program not expressly granted under
-this Agreement are reserved.
-
-This Agreement is governed by the laws of the State of New York and the
-intellectual property laws of the United States of America. No party to this
-Agreement will bring a legal action under this Agreement more than one year
-after the cause of action arose. Each party waives its rights to a jury trial in
-any resulting litigation.
-
-The binary distribution of this product bundles these dependencies under the
-following license:
-ASM Core 3.2
-JSch 0.1.51
-ParaNamer Core 2.3
-JLine 0.9.94
-leveldbjni-all 1.8
-Hamcrest Core 1.3
-xmlenc Library 0.52
---------------------------------------------------------------------------------
-Redistribution and use in source and binary forms, with or without
-modification, are permitted provided that the following conditions are met:
-    * Redistributions of source code must retain the above copyright
-      notice, this list of conditions and the following disclaimer.
-    * Redistributions in binary form must reproduce the above copyright
-      notice, this list of conditions and the following disclaimer in the
-      documentation and/or other materials provided with the distribution.
-    * Neither the name of the <organization> nor the
-      names of its contributors may be used to endorse or promote products
-      derived from this software without specific prior written permission.
-
-THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" AND
-ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED
-WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE
-DISCLAIMED. IN NO EVENT SHALL <COPYRIGHT HOLDER> BE LIABLE FOR ANY
-DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES
-(INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES;
-LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND
-ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
-(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS
-SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
-
-The binary distribution of this product bundles these dependencies under the
-following license:
-FindBugs-jsr305 3.0.0
---------------------------------------------------------------------------------
-Redistribution and use in source and binary forms, with or without
-modification, are permitted provided that the following conditions are met:
-
-1. Redistributions of source code must retain the above copyright notice, this
-   list of conditions and the following disclaimer.
-2. Redistributions in binary form must reproduce the above copyright notice,
-   this list of conditions and the following disclaimer in the documentation
-   and/or other materials provided with the distribution.
-
-THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" AND
-ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED
-WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE
-DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE LIABLE FOR
-ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES
-(INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES;
-LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND
-ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
-(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS
-SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
-
-The views and conclusions contained in the software and documentation are those
-of the authors and should not be interpreted as representing official policies,
-either expressed or implied, of the FreeBSD Project.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ee55b74f/NOTICE.txt
----------------------------------------------------------------------
diff --git a/NOTICE.txt b/NOTICE.txt
index 63fbc9d..efa865f 100644
--- a/NOTICE.txt
+++ b/NOTICE.txt
@@ -15,269 +15,3 @@ which has the following notices:
 * This product includes software developed at
   Progress Software Corporation and/or its  subsidiaries or affiliates.
 * This product includes software developed by IBM Corporation and others.
-
-The binary distribution of this product bundles binaries of
-AWS Java SDK 1.10.6,
-which has the following notices:
- * This software includes third party software subject to the following
- copyrights: - XML parsing and utility functions from JetS3t - Copyright
- 2006-2009 James Murty. - JSON parsing and utility functions from JSON.org -
- Copyright 2002 JSON.org. - PKCS#1 PEM encoded private key parsing and utility
- functions from oauth.googlecode.com - Copyright 1998-2010 AOL Inc.
-
-The binary distribution of this product bundles binaries of
-Gson 2.2.4,
-which has the following notices:
-
-                            The Netty Project
-                            =================
-
-Please visit the Netty web site for more information:
-
-  * http://netty.io/
-
-Copyright 2014 The Netty Project
-
-The Netty Project 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.
-
-Also, please refer to each LICENSE.<component>.txt file, which is located in
-the 'license' directory of the distribution file, for the license terms of the
-components that this product depends on.
-
--------------------------------------------------------------------------------
-This product contains the extensions to Java Collections Framework which has
-been derived from the works by JSR-166 EG, Doug Lea, and Jason T. Greene:
-
-  * LICENSE:
-    * license/LICENSE.jsr166y.txt (Public Domain)
-  * HOMEPAGE:
-    * http://gee.cs.oswego.edu/cgi-bin/viewcvs.cgi/jsr166/
-    * http://viewvc.jboss.org/cgi-bin/viewvc.cgi/jbosscache/experimental/jsr166/
-
-This product contains a modified version of Robert Harder's Public Domain
-Base64 Encoder and Decoder, which can be obtained at:
-
-  * LICENSE:
-    * license/LICENSE.base64.txt (Public Domain)
-  * HOMEPAGE:
-    * http://iharder.sourceforge.net/current/java/base64/
-
-This product contains a modified portion of 'Webbit', an event based
-WebSocket and HTTP server, which can be obtained at:
-
-  * LICENSE:
-    * license/LICENSE.webbit.txt (BSD License)
-  * HOMEPAGE:
-    * https://github.com/joewalnes/webbit
-
-This product contains a modified portion of 'SLF4J', a simple logging
-facade for Java, which can be obtained at:
-
-  * LICENSE:
-    * license/LICENSE.slf4j.txt (MIT License)
-  * HOMEPAGE:
-    * http://www.slf4j.org/
-
-This product contains a modified portion of 'ArrayDeque', written by Josh
-Bloch of Google, Inc:
-
-  * LICENSE:
-    * license/LICENSE.deque.txt (Public Domain)
-
-This product contains a modified portion of 'Apache Harmony', an open source
-Java SE, which can be obtained at:
-
-  * LICENSE:
-    * license/LICENSE.harmony.txt (Apache License 2.0)
-  * HOMEPAGE:
-    * http://archive.apache.org/dist/harmony/
-
-This product contains a modified version of Roland Kuhn's ASL2
-AbstractNodeQueue, which is based on Dmitriy Vyukov's non-intrusive MPSC queue.
-It can be obtained at:
-
-  * LICENSE:
-    * license/LICENSE.abstractnodequeue.txt (Public Domain)
-  * HOMEPAGE:
-    * https://github.com/akka/akka/blob/wip-2.2.3-for-scala-2.11/akka-actor/src/main/java/akka/dispatch/AbstractNodeQueue.java
-
-This product contains a modified portion of 'jbzip2', a Java bzip2 compression
-and decompression library written by Matthew J. Francis. It can be obtained at:
-
-  * LICENSE:
-    * license/LICENSE.jbzip2.txt (MIT License)
-  * HOMEPAGE:
-    * https://code.google.com/p/jbzip2/
-
-This product contains a modified portion of 'libdivsufsort', a C API library to construct
-the suffix array and the Burrows-Wheeler transformed string for any input string of
-a constant-size alphabet written by Yuta Mori. It can be obtained at:
-
-  * LICENSE:
-    * license/LICENSE.libdivsufsort.txt (MIT License)
-  * HOMEPAGE:
-    * https://code.google.com/p/libdivsufsort/
-
-This product contains a modified portion of Nitsan Wakart's 'JCTools', Java Concurrency Tools for the JVM,
- which can be obtained at:
-
-  * LICENSE:
-    * license/LICENSE.jctools.txt (ASL2 License)
-  * HOMEPAGE:
-    * https://github.com/JCTools/JCTools
-
-This product optionally depends on 'JZlib', a re-implementation of zlib in
-pure Java, which can be obtained at:
-
-  * LICENSE:
-    * license/LICENSE.jzlib.txt (BSD style License)
-  * HOMEPAGE:
-    * http://www.jcraft.com/jzlib/
-
-This product optionally depends on 'Compress-LZF', a Java library for encoding and
-decoding data in LZF format, written by Tatu Saloranta. It can be obtained at:
-
-  * LICENSE:
-    * license/LICENSE.compress-lzf.txt (Apache License 2.0)
-  * HOMEPAGE:
-    * https://github.com/ning/compress
-
-This product optionally depends on 'lz4', a LZ4 Java compression
-and decompression library written by Adrien Grand. It can be obtained at:
-
-  * LICENSE:
-    * license/LICENSE.lz4.txt (Apache License 2.0)
-  * HOMEPAGE:
-    * https://github.com/jpountz/lz4-java
-
-This product optionally depends on 'lzma-java', a LZMA Java compression
-and decompression library, which can be obtained at:
-
-  * LICENSE:
-    * license/LICENSE.lzma-java.txt (Apache License 2.0)
-  * HOMEPAGE:
-    * https://github.com/jponge/lzma-java
-
-This product contains a modified portion of 'jfastlz', a Java port of FastLZ compression
-and decompression library written by William Kinney. It can be obtained at:
-
-  * LICENSE:
-    * license/LICENSE.jfastlz.txt (MIT License)
-  * HOMEPAGE:
-    * https://code.google.com/p/jfastlz/
-
-This product contains a modified portion of and optionally depends on 'Protocol Buffers', Google's data
-interchange format, which can be obtained at:
-
-  * LICENSE:
-    * license/LICENSE.protobuf.txt (New BSD License)
-  * HOMEPAGE:
-    * http://code.google.com/p/protobuf/
-
-This product optionally depends on 'Bouncy Castle Crypto APIs' to generate
-a temporary self-signed X.509 certificate when the JVM does not provide the
-equivalent functionality.  It can be obtained at:
-
-  * LICENSE:
-    * license/LICENSE.bouncycastle.txt (MIT License)
-  * HOMEPAGE:
-    * http://www.bouncycastle.org/
-
-This product optionally depends on 'Snappy', a compression library produced
-by Google Inc, which can be obtained at:
-
-  * LICENSE:
-    * license/LICENSE.snappy.txt (New BSD License)
-  * HOMEPAGE:
-    * http://code.google.com/p/snappy/
-
-This product optionally depends on 'JBoss Marshalling', an alternative Java
-serialization API, which can be obtained at:
-
-  * LICENSE:
-    * license/LICENSE.jboss-marshalling.txt (GNU LGPL 2.1)
-  * HOMEPAGE:
-    * http://www.jboss.org/jbossmarshalling
-
-This product optionally depends on 'Caliper', Google's micro-
-benchmarking framework, which can be obtained at:
-
-  * LICENSE:
-    * license/LICENSE.caliper.txt (Apache License 2.0)
-  * HOMEPAGE:
-    * http://code.google.com/p/caliper/
-
-This product optionally depends on 'Apache Commons Logging', a logging
-framework, which can be obtained at:
-
-  * LICENSE:
-    * license/LICENSE.commons-logging.txt (Apache License 2.0)
-  * HOMEPAGE:
-    * http://commons.apache.org/logging/
-
-This product optionally depends on 'Apache Log4J', a logging framework, which
-can be obtained at:
-
-  * LICENSE:
-    * license/LICENSE.log4j.txt (Apache License 2.0)
-  * HOMEPAGE:
-    * http://logging.apache.org/log4j/
-
-This product optionally depends on 'Aalto XML', an ultra-high performance
-non-blocking XML processor, which can be obtained at:
-
-  * LICENSE:
-    * license/LICENSE.aalto-xml.txt (Apache License 2.0)
-  * HOMEPAGE:
-    * http://wiki.fasterxml.com/AaltoHome
-
-This product contains a modified version of 'HPACK', a Java implementation of
-the HTTP/2 HPACK algorithm written by Twitter. It can be obtained at:
-
-  * LICENSE:
-    * license/LICENSE.hpack.txt (Apache License 2.0)
-  * HOMEPAGE:
-    * https://github.com/twitter/hpack
-
-This product contains a modified portion of 'Apache Commons Lang', a Java library
-provides utilities for the java.lang API, which can be obtained at:
-
-  * LICENSE:
-    * license/LICENSE.commons-lang.txt (Apache License 2.0)
-  * HOMEPAGE:
-    * https://commons.apache.org/proper/commons-lang/
-
-The binary distribution of this product bundles binaries of
-Commons Codec 1.4,
-which has the following notices:
- * src/test/org/apache/commons/codec/language/DoubleMetaphoneTest.javacontains test data from http://aspell.net/test/orig/batch0.tab.Copyright (C) 2002 Kevin Atkinson (kevina@gnu.org)
-  ===============================================================================
-  The content of package org.apache.commons.codec.language.bm has been translated
-  from the original php source code available at http://stevemorse.org/phoneticinfo.htm
-  with permission from the original authors.
-  Original source copyright:Copyright (c) 2008 Alexander Beider & Stephen P. Morse.
-
-The binary distribution of this product bundles binaries of
-Commons Lang 2.6,
-which has the following notices:
- * This product includes software from the Spring Framework,under the Apache License 2.0 (see: StringUtils.containsWhitespace())
-
-The binary distribution of this product bundles binaries of
-Apache Log4j 1.2.17,
-which has the following notices:
- * ResolverUtil.java
-    Copyright 2005-2006 Tim Fennell
-  Dumbster SMTP test server
-    Copyright 2004 Jason Paul Kitchen
-  TypeUtil.java
-    Copyright 2002-2012 Ramnivas Laddad, Juergen Hoeller, Chris Beams
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ee55b74f/hadoop-build-tools/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-build-tools/pom.xml b/hadoop-build-tools/pom.xml
index e84c94f..d08cb89 100644
--- a/hadoop-build-tools/pom.xml
+++ b/hadoop-build-tools/pom.xml
@@ -28,45 +28,4 @@
   <properties>
     <failIfNoTests>false</failIfNoTests>
   </properties>
-  <build>
-    <plugins>
-      <plugin>
-        <groupId>org.apache.maven.plugins</groupId>
-        <artifactId>maven-site-plugin</artifactId>
-        <configuration>
-          <skip>true</skip>
-        </configuration>
-      </plugin>
-      <plugin>
-        <groupId>org.apache.maven.plugins</groupId>
-        <artifactId>maven-remote-resources-plugin</artifactId>
-        <executions>
-          <execution>
-            <goals>
-              <goal>bundle</goal>
-            </goals>
-          </execution>
-        </executions>
-        <configuration>
-          <includes>
-            <include>META-INF/LICENSE.txt</include>
-            <include>META-INF/NOTICE.txt</include>
-          </includes>
-        </configuration>
-      </plugin>
-      <plugin>
-        <groupId>org.apache.maven.plugins</groupId>
-        <artifactId>maven-antrun-plugin</artifactId>
-        <executions>
-          <execution>
-            <id>dummy</id>
-            <phase>validate</phase>
-            <goals>
-              <goal>run</goal>
-            </goals>
-          </execution>
-        </executions>
-      </plugin>
-    </plugins>
-  </build>
 </project>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ee55b74f/hadoop-project-dist/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-project-dist/pom.xml b/hadoop-project-dist/pom.xml
index ba4c0a8..2cccbbe 100644
--- a/hadoop-project-dist/pom.xml
+++ b/hadoop-project-dist/pom.xml
@@ -80,8 +80,6 @@
               <includes>
                 <include>**/*.class</include>
                 <include>webapps/**</include>
-                <include>META-INF/LICENSE.txt</include>
-                <include>META-INF/NOTICE.txt</include>
               </includes>
             </configuration>
           </execution>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ee55b74f/hadoop-project/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-project/pom.xml b/hadoop-project/pom.xml
index 7558b04..35166b1 100644
--- a/hadoop-project/pom.xml
+++ b/hadoop-project/pom.xml
@@ -111,7 +111,6 @@
     <maven-source-plugin.version>2.3</maven-source-plugin.version>
     <maven-project-info-reports-plugin.version>2.7</maven-project-info-reports-plugin.version>
     <maven-pdf-plugin.version>1.2</maven-pdf-plugin.version>
-    <maven-remote-resources-plugin.version>1.5</maven-remote-resources-plugin.version>
     <build-helper-maven-plugin.version>1.9</build-helper-maven-plugin.version>
     <exec-maven-plugin.version>1.3.1</exec-maven-plugin.version>
     <make-maven-plugin.version>1.0-beta-1</make-maven-plugin.version>
@@ -1135,23 +1134,7 @@
         <groupId>org.codehaus.mojo</groupId>
         <artifactId>findbugs-maven-plugin</artifactId>
       </plugin>
-      <plugin>
-        <groupId>org.apache.maven.plugins</groupId>
-        <artifactId>maven-remote-resources-plugin</artifactId>
-        <version>${maven-remote-resources-plugin.version}</version>
-        <configuration>
-          <resourceBundles>
-            <resourceBundle>org.apache.hadoop:hadoop-build-tools:${project.version}</resourceBundle>
-          </resourceBundles>
-        </configuration>
-        <executions>
-          <execution>
-            <goals>
-              <goal>process</goal>
-            </goals>
-          </execution>
-        </executions>
-      </plugin>
+      
        <plugin>
          <groupId>org.apache.maven.plugins</groupId>
         <artifactId>maven-antrun-plugin</artifactId>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ee55b74f/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index aec6722..c908340 100644
--- a/pom.xml
+++ b/pom.xml
@@ -412,32 +412,6 @@ xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xs
         <artifactId>dependency-check-maven</artifactId>
         <version>${dependency-check-maven.version}</version>
       </plugin>
-      <plugin>
-        <!-- Copy license and notice files into hadoop-build-tools'
-             metadata, which will then be bundled into jars.
-        -->
-        <groupId>org.apache.maven.plugins</groupId>
-        <artifactId>maven-antrun-plugin</artifactId>
-        <executions>
-          <execution>
-            <id>copy-files</id>
-            <phase>process-resources</phase>
-            <goals>
-              <goal>run</goal>
-            </goals>
-            <configuration>
-              <target>
-                <copy todir="${project.build.directory}/../hadoop-build-tools/src/main/resources/META-INF/">
-                  <fileset dir="${project.build.directory}/../">
-                    <include name="LICENSE.txt"/>
-                    <include name="NOTICE.txt"/>
-                  </fileset>
-                </copy>
-              </target>
-            </configuration>
-          </execution>
-        </executions>
-      </plugin>
     </plugins>
   </build>
 


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


[34/39] hadoop git commit: HADOOP-13285. DecayRpcScheduler MXBean should only report decayed CallVolumeSummary. Contributed by Xiaoyu Yao.

Posted by ae...@apache.org.
HADOOP-13285. DecayRpcScheduler MXBean should only report decayed CallVolumeSummary. Contributed by Xiaoyu Yao.


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

Branch: refs/heads/HDFS-1312
Commit: 0761379fe45898c44c8f161834c298ef932e4d8c
Parents: 2800695
Author: Xiaoyu Yao <xy...@apache.org>
Authored: Fri Jun 17 15:25:14 2016 -0700
Committer: Xiaoyu Yao <xy...@apache.org>
Committed: Fri Jun 17 15:25:14 2016 -0700

----------------------------------------------------------------------
 .../apache/hadoop/ipc/DecayRpcScheduler.java    | 17 ++++++++++-
 .../hadoop/ipc/TestDecayRpcScheduler.java       | 30 ++++++++++++++++----
 2 files changed, 41 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/0761379f/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/DecayRpcScheduler.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/DecayRpcScheduler.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/DecayRpcScheduler.java
index ec87c75..f40bd17 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/DecayRpcScheduler.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/DecayRpcScheduler.java
@@ -901,9 +901,24 @@ public class DecayRpcScheduler implements RpcScheduler,
   public String getCallVolumeSummary() {
     try {
       ObjectMapper om = new ObjectMapper();
-      return om.writeValueAsString(callCounts);
+      return om.writeValueAsString(getDecayedCallCounts());
     } catch (Exception e) {
       return "Error: " + e.getMessage();
     }
   }
+
+  private Map<Object, Long> getDecayedCallCounts() {
+    Map<Object, Long> decayedCallCounts = new HashMap<>(callCounts.size());
+    Iterator<Map.Entry<Object, List<AtomicLong>>> it =
+        callCounts.entrySet().iterator();
+    while (it.hasNext()) {
+      Map.Entry<Object, List<AtomicLong>> entry = it.next();
+      Object user = entry.getKey();
+      Long decayedCount = entry.getValue().get(0).get();
+      if (decayedCount > 0) {
+        decayedCallCounts.put(user, decayedCount);
+      }
+    }
+    return decayedCallCounts;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0761379f/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestDecayRpcScheduler.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestDecayRpcScheduler.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestDecayRpcScheduler.java
index 0b0408c..58380c5 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestDecayRpcScheduler.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestDecayRpcScheduler.java
@@ -30,6 +30,10 @@ import static org.mockito.Mockito.when;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.conf.Configuration;
 
+import javax.management.MBeanServer;
+import javax.management.ObjectName;
+import java.lang.management.ManagementFactory;
+
 public class TestDecayRpcScheduler {
   private Schedulable mockCall(String id) {
     Schedulable mockCall = mock(Schedulable.class);
@@ -189,12 +193,14 @@ public class TestDecayRpcScheduler {
 
   @Test
   @SuppressWarnings("deprecation")
-  public void testPriority() {
+  public void testPriority() throws Exception {
     Configuration conf = new Configuration();
-    conf.set("ns." + DecayRpcScheduler.IPC_FCQ_DECAYSCHEDULER_PERIOD_KEY, "99999999"); // Never flush
-    conf.set("ns." + DecayRpcScheduler.IPC_FCQ_DECAYSCHEDULER_THRESHOLDS_KEY,
-      "25, 50, 75");
-    scheduler = new DecayRpcScheduler(4, "ns", conf);
+    final String namespace = "ns";
+    conf.set(namespace + "." + DecayRpcScheduler
+        .IPC_FCQ_DECAYSCHEDULER_PERIOD_KEY, "99999999"); // Never flush
+    conf.set(namespace + "." + DecayRpcScheduler
+        .IPC_FCQ_DECAYSCHEDULER_THRESHOLDS_KEY, "25, 50, 75");
+    scheduler = new DecayRpcScheduler(4, namespace, conf);
 
     assertEquals(0, scheduler.getPriorityLevel(mockCall("A")));
     assertEquals(2, scheduler.getPriorityLevel(mockCall("A")));
@@ -206,6 +212,20 @@ public class TestDecayRpcScheduler {
     assertEquals(1, scheduler.getPriorityLevel(mockCall("A")));
     assertEquals(1, scheduler.getPriorityLevel(mockCall("A")));
     assertEquals(2, scheduler.getPriorityLevel(mockCall("A")));
+
+    MBeanServer mbs = ManagementFactory.getPlatformMBeanServer();
+    ObjectName mxbeanName = new ObjectName(
+        "Hadoop:service="+ namespace + ",name=DecayRpcScheduler");
+
+    String cvs1 = (String) mbs.getAttribute(mxbeanName, "CallVolumeSummary");
+    assertTrue("Get expected JMX of CallVolumeSummary before decay",
+        cvs1.equals("{\"A\":6,\"B\":2,\"C\":2}"));
+
+    scheduler.forceDecay();
+
+    String cvs2 = (String) mbs.getAttribute(mxbeanName, "CallVolumeSummary");
+    assertTrue("Get expected JMX for CallVolumeSummary after decay",
+        cvs2.equals("{\"A\":3,\"B\":1,\"C\":1}"));
   }
 
   @Test(timeout=2000)


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


[30/39] hadoop git commit: HADOOP-13242. Authenticate to Azure Data Lake using client ID and keys. Contributed by Atul Sikaria.

Posted by ae...@apache.org.
HADOOP-13242. Authenticate to Azure Data Lake using client ID and keys. Contributed by Atul Sikaria.


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

Branch: refs/heads/HDFS-1312
Commit: 51d16e7b38d247f73b0ec2ffd8b2b02069c05a33
Parents: 51d497f
Author: Chris Nauroth <cn...@apache.org>
Authored: Thu Jun 16 23:35:20 2016 -0700
Committer: Chris Nauroth <cn...@apache.org>
Committed: Thu Jun 16 23:35:20 2016 -0700

----------------------------------------------------------------------
 hadoop-tools/hadoop-azure-datalake/pom.xml      |   7 +-
 ...ClientCredentialBasedAccesTokenProvider.java | 155 +++++++++++++++++++
 .../src/site/markdown/index.md                  |  64 ++++++++
 3 files changed, 225 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/51d16e7b/hadoop-tools/hadoop-azure-datalake/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure-datalake/pom.xml b/hadoop-tools/hadoop-azure-datalake/pom.xml
index a4b1fe1..d2161c7 100644
--- a/hadoop-tools/hadoop-azure-datalake/pom.xml
+++ b/hadoop-tools/hadoop-azure-datalake/pom.xml
@@ -147,7 +147,12 @@
     <dependency>
     <groupId>org.apache.hadoop</groupId>
       <artifactId>hadoop-common</artifactId>
-  </dependency>
+    </dependency>
+    <dependency>
+      <groupId>com.squareup.okhttp</groupId>
+      <artifactId>okhttp</artifactId>
+      <version>2.4.0</version>
+    </dependency>
     <dependency>
       <groupId>junit</groupId>
       <artifactId>junit</artifactId>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/51d16e7b/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/hdfs/web/oauth2/AzureADClientCredentialBasedAccesTokenProvider.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/hdfs/web/oauth2/AzureADClientCredentialBasedAccesTokenProvider.java b/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/hdfs/web/oauth2/AzureADClientCredentialBasedAccesTokenProvider.java
new file mode 100644
index 0000000..6dfc593
--- /dev/null
+++ b/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/hdfs/web/oauth2/AzureADClientCredentialBasedAccesTokenProvider.java
@@ -0,0 +1,155 @@
+/*
+ * 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.web.oauth2;
+
+import com.squareup.okhttp.OkHttpClient;
+import com.squareup.okhttp.Request;
+import com.squareup.okhttp.RequestBody;
+import com.squareup.okhttp.Response;
+import com.squareup.okhttp.MediaType;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.web.URLConnectionFactory;
+import org.apache.hadoop.util.Timer;
+import org.apache.http.HttpStatus;
+import org.codehaus.jackson.map.ObjectMapper;
+import org.codehaus.jackson.map.ObjectReader;
+
+import java.io.IOException;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+
+import static org.apache.hadoop.hdfs.web.oauth2.Utils.notNull;
+
+
+/**
+ * Obtain an access token via the credential-based OAuth2 workflow.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public class AzureADClientCredentialBasedAccesTokenProvider
+    extends AccessTokenProvider {
+  private static final ObjectReader READER =
+      new ObjectMapper().reader(Map.class);
+
+  public static final String OAUTH_CREDENTIAL_KEY
+      = "dfs.webhdfs.oauth2.credential";
+
+  public static final String AAD_RESOURCE_KEY
+      = "fs.adls.oauth2.resource";
+
+  public static final String RESOURCE_PARAM_NAME
+      = "resource";
+
+  private static final String OAUTH_CLIENT_ID_KEY
+      = "dfs.webhdfs.oauth2.client.id";
+
+  private static final String OAUTH_REFRESH_URL_KEY
+      = "dfs.webhdfs.oauth2.refresh.url";
+
+
+  public static final String ACCESS_TOKEN = "access_token";
+  public static final String CLIENT_CREDENTIALS = "client_credentials";
+  public static final String CLIENT_ID = "client_id";
+  public static final String CLIENT_SECRET = "client_secret";
+  public static final String EXPIRES_IN = "expires_in";
+  public static final String GRANT_TYPE = "grant_type";
+  public static final MediaType URLENCODED
+          = MediaType.parse("application/x-www-form-urlencoded; charset=utf-8");
+
+
+  private AccessTokenTimer timer;
+
+  private String clientId;
+
+  private String refreshURL;
+
+  private String accessToken;
+
+  private String resource;
+
+  private String credential;
+
+  private boolean initialCredentialObtained = false;
+
+  AzureADClientCredentialBasedAccesTokenProvider() {
+    this.timer = new AccessTokenTimer();
+  }
+
+  AzureADClientCredentialBasedAccesTokenProvider(Timer timer) {
+    this.timer = new AccessTokenTimer(timer);
+  }
+
+  @Override
+  public void setConf(Configuration conf) {
+    super.setConf(conf);
+    clientId = notNull(conf, OAUTH_CLIENT_ID_KEY);
+    refreshURL = notNull(conf, OAUTH_REFRESH_URL_KEY);
+    resource = notNull(conf, AAD_RESOURCE_KEY);
+    credential = notNull(conf, OAUTH_CREDENTIAL_KEY);
+  }
+
+  @Override
+  public String getAccessToken() throws IOException {
+    if(timer.shouldRefresh() || !initialCredentialObtained) {
+      refresh();
+      initialCredentialObtained = true;
+    }
+    return accessToken;
+  }
+
+  void refresh() throws IOException {
+    try {
+      OkHttpClient client = new OkHttpClient();
+      client.setConnectTimeout(URLConnectionFactory.DEFAULT_SOCKET_TIMEOUT,
+          TimeUnit.MILLISECONDS);
+      client.setReadTimeout(URLConnectionFactory.DEFAULT_SOCKET_TIMEOUT,
+          TimeUnit.MILLISECONDS);
+
+      String bodyString = Utils.postBody(CLIENT_SECRET, credential,
+          GRANT_TYPE, CLIENT_CREDENTIALS,
+          RESOURCE_PARAM_NAME, resource,
+          CLIENT_ID, clientId);
+
+      RequestBody body = RequestBody.create(URLENCODED, bodyString);
+
+      Request request = new Request.Builder()
+          .url(refreshURL)
+          .post(body)
+          .build();
+      Response responseBody = client.newCall(request).execute();
+
+      if (responseBody.code() != HttpStatus.SC_OK) {
+        throw new IllegalArgumentException("Received invalid http response: "
+            + responseBody.code() + ", text = " + responseBody.toString());
+      }
+
+      Map<?, ?> response = READER.readValue(responseBody.body().string());
+
+      String newExpiresIn = response.get(EXPIRES_IN).toString();
+      timer.setExpiresIn(newExpiresIn);
+
+      accessToken = response.get(ACCESS_TOKEN).toString();
+
+    } catch (Exception e) {
+      throw new IOException("Unable to obtain access token from credential", e);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/51d16e7b/hadoop-tools/hadoop-azure-datalake/src/site/markdown/index.md
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure-datalake/src/site/markdown/index.md b/hadoop-tools/hadoop-azure-datalake/src/site/markdown/index.md
index 4158c88..3f03d41 100644
--- a/hadoop-tools/hadoop-azure-datalake/src/site/markdown/index.md
+++ b/hadoop-tools/hadoop-azure-datalake/src/site/markdown/index.md
@@ -23,6 +23,9 @@
         * [OAuth2 Support](#OAuth2_Support)
         * [Read Ahead Buffer Management](Read_Ahead_Buffer_Management)
     * [Configuring Credentials & FileSystem](#Configuring_Credentials)
+        * [Using Refresh Token](#Refresh_Token)
+        * [Using Client Keys](#Client_Credential_Token)
+    * [Enabling ADL Filesystem](#Enabling_ADL)
     * [Accessing adl URLs](#Accessing_adl_URLs)
 * [Testing the hadoop-azure Module](#Testing_the_hadoop-azure_Module)
 
@@ -131,6 +134,9 @@ To configure number of concurrent connection to Azure Data Lake Storage Account.
     </property>
 
 ## <a name="Configuring_Credentials" />Configuring Credentials & FileSystem
+Credentials can be configured using either a refresh token (associated with a user) or a client credential (analogous to a service principal).
+
+### <a name="Refresh_Token" />Using Refresh Token
 
 Update core-site.xml for OAuth2 configuration
 
@@ -173,6 +179,64 @@ Application require to set Client id and OAuth2 refresh token from Azure Active
             <value></value>
         </property>
 
+
+### <a name="Client_Credential_Token" />Using Client Keys
+
+#### Generating the Service Principal
+1.  Go to the portal (https://portal.azure.com)
+2.  Under "Browse", look for Active Directory and click on it.
+3.  Create "Web Application". Remember the name you create here - that is what you will add to your ADL account as authorized user.
+4.  Go through the wizard
+5.  Once app is created, Go to app configuration, and find the section on "keys"
+6.  Select a key duration and hit save. Save the generated keys.
+7. Note down the properties you will need to auth:
+    -  The client ID
+    -  The key you just generated above
+    -  The token endpoint (select "View endpoints" at the bottom of the page and copy/paste the OAuth2 .0 Token Endpoint value)
+    -  Resource: Always https://management.core.windows.net/ , for all customers
+
+#### Adding the service principal to your ADL Account
+1.  Go to the portal again, and open your ADL account
+2.  Select Users under Settings
+3.  Add your user name you created in Step 6 above (note that it does not show up in the list, but will be found if you searched for the name)
+4.  Add "Owner" role
+
+#### Configure core-site.xml
+Add the following properties to your core-site.xml
+
+    <property>
+      <name>dfs.webhdfs.oauth2.access.token.provider</name>
+      <value>org.apache.hadoop.hdfs.web.oauth2.AzureADClientCredentialBasedAccesTokenProvider</value>
+    </property>
+
+    <property>
+      <name>dfs.webhdfs.oauth2.refresh.url</name>
+      <value>TOKEN ENDPOINT FROM STEP 7 ABOVE</value>
+    </property>
+
+    <property>
+      <name>dfs.webhdfs.oauth2.client.id</name>
+      <value>CLIENT ID FROM STEP 7 ABOVE</value>
+    </property>
+
+    <property>
+      <name>dfs.webhdfs.oauth2.credential</name>
+      <value>PASSWORD FROM STEP 7 ABOVE</value>
+    </property>
+
+    <property>
+      <name>fs.adls.oauth2.resource</name>
+      <value>https://management.core.windows.net/</value>
+    </property>
+
+    <property>
+      <name>fs.defaultFS</name>
+      <value>YOUR ADL STORE URL (e.g., https://example.azuredatalakestore.net) </value>
+    </property>
+
+
+## <a name="Enabling_ADL" />Enabling ADL Filesystem
+
 For ADL FileSystem to take effect. Update core-site.xml with
 
         <property>


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


[23/39] hadoop git commit: HDFS-10532. Typo in RollingUpgrade docs. (Contributed by Yiqun Lin)

Posted by ae...@apache.org.
HDFS-10532. Typo in RollingUpgrade docs. (Contributed by Yiqun Lin)


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

Branch: refs/heads/HDFS-1312
Commit: e983eafb44a676514af31bb16bbb654104938cbd
Parents: c9e7138
Author: Arpit Agarwal <ar...@apache.org>
Authored: Thu Jun 16 10:10:33 2016 -0700
Committer: Arpit Agarwal <ar...@apache.org>
Committed: Thu Jun 16 10:10:33 2016 -0700

----------------------------------------------------------------------
 .../hadoop-hdfs/src/site/markdown/HdfsRollingUpgrade.md            | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/e983eafb/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HdfsRollingUpgrade.md
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HdfsRollingUpgrade.md b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HdfsRollingUpgrade.md
index 5415912..7514c49 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HdfsRollingUpgrade.md
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HdfsRollingUpgrade.md
@@ -172,7 +172,7 @@ Then, the files created before or after *T* remain available in HDFS.
 The files deleted before or after *T* remain deleted in HDFS.
 
 A newer release is downgradable to the pre-upgrade release
-only if both the namenode layout version and the datenode layout version
+only if both the namenode layout version and the datanode layout version
 are not changed between these two releases.
 
 In a HA cluster,


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


[20/39] hadoop git commit: YARN-5083. YARN CLI for AM logs does not give any error message if entered invalid am value. Contributed by Jian He.

Posted by ae...@apache.org.
YARN-5083. YARN CLI for AM logs does not give any error message if entered invalid am value. 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/e14ee0d3
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/e14ee0d3
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/e14ee0d3

Branch: refs/heads/HDFS-1312
Commit: e14ee0d3b55816bed1d27a8caf78001985119e3c
Parents: cc6c265
Author: Junping Du <ju...@apache.org>
Authored: Thu Jun 16 08:55:56 2016 -0700
Committer: Junping Du <ju...@apache.org>
Committed: Thu Jun 16 08:55:56 2016 -0700

----------------------------------------------------------------------
 .../apache/hadoop/yarn/client/cli/LogsCLI.java  |  7 +++++-
 .../hadoop/yarn/client/cli/TestLogsCLI.java     | 23 ++++++++++++++++++++
 2 files changed, 29 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/e14ee0d3/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/LogsCLI.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/LogsCLI.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/LogsCLI.java
index ebd5428..d28c77c 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/LogsCLI.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/LogsCLI.java
@@ -274,7 +274,7 @@ public class LogsCLI extends Configured implements Tool {
     formatter.printHelp("general options are:", options);
   }
 
-  private List<JSONObject> getAMContainerInfoForRMWebService(
+  protected List<JSONObject> getAMContainerInfoForRMWebService(
       Configuration conf, String appId) throws ClientHandlerException,
       UniformInterfaceException, JSONException {
     Client webServiceClient = Client.create();
@@ -538,6 +538,11 @@ public class LogsCLI extends Configured implements Tool {
           if (amContainerId <= requests.size()) {
             outputAMContainerLogs(requests.get(amContainerId - 1), conf,
                 logCliHelper);
+          } else {
+            System.err.println(String.format("ERROR: Specified AM containerId"
+                + " (%s) exceeds the number of AM containers (%s).",
+                amContainerId, requests.size()));
+            return -1;
           }
         }
       }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e14ee0d3/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestLogsCLI.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestLogsCLI.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestLogsCLI.java
index 10047e3..df1c6b7 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestLogsCLI.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestLogsCLI.java
@@ -71,6 +71,7 @@ import org.apache.hadoop.yarn.logaggregation.AggregatedLogFormat;
 import org.apache.hadoop.yarn.logaggregation.ContainerLogsRequest;
 import org.apache.hadoop.yarn.logaggregation.LogAggregationUtils;
 import org.apache.hadoop.yarn.logaggregation.LogCLIHelpers;
+import org.codehaus.jettison.json.JSONObject;
 import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
@@ -131,6 +132,28 @@ public class TestLogsCLI {
     assertTrue(sysErrStream.toString().startsWith("Invalid ApplicationId specified"));
   }
 
+  @Test(timeout = 5000L)
+  public void testInvalidAMContainerId() throws Exception {
+    Configuration conf = new YarnConfiguration();
+    conf.setBoolean(YarnConfiguration.APPLICATION_HISTORY_ENABLED, true);
+    YarnClient mockYarnClient =
+        createMockYarnClient(YarnApplicationState.FINISHED,
+            UserGroupInformation.getCurrentUser().getShortUserName());
+    LogsCLI cli = spy(new LogsCLIForTest(mockYarnClient));
+    List<JSONObject> list = Arrays.asList(new JSONObject());
+    doReturn(list).when(cli)
+        .getAMContainerInfoForRMWebService(any(Configuration.class),
+            any(String.class));
+    cli.setConf(conf);
+
+    int exitCode = cli.run(
+        new String[] {"-applicationId", "application_1465862913885_0027",
+            "-am", "1000" });
+    assertTrue(exitCode == -1);
+    assertTrue(sysErrStream.toString()
+        .contains("exceeds the number of AM containers"));
+  }
+
   @Test(timeout = 5000l)
   public void testUnknownApplicationId() throws Exception {
     Configuration conf = new YarnConfiguration();


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


[36/39] hadoop git commit: YARN-5077. Fix FSLeafQueue#getFairShare() for queues with zero fairshare. (Yufei Gu via kasha)

Posted by ae...@apache.org.
YARN-5077. Fix FSLeafQueue#getFairShare() for queues with zero fairshare. (Yufei Gu via kasha)


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

Branch: refs/heads/HDFS-1312
Commit: 20f2799938c5cad59f59617cdb0f86e137c3c2c0
Parents: fbbe0bb
Author: Karthik Kambatla <ka...@apache.org>
Authored: Fri Jun 17 22:24:42 2016 -0700
Committer: Karthik Kambatla <ka...@apache.org>
Committed: Fri Jun 17 22:24:42 2016 -0700

----------------------------------------------------------------------
 .../scheduler/fair/FSLeafQueue.java             | 33 ++++-----
 .../resourcemanager/scheduler/fair/FSQueue.java | 19 -----
 .../scheduler/fair/SchedulingPolicy.java        | 11 ---
 .../DominantResourceFairnessPolicy.java         |  5 --
 .../fair/policies/FairSharePolicy.java          |  5 --
 .../scheduler/fair/policies/FifoPolicy.java     |  5 --
 .../scheduler/fair/TestFairScheduler.java       | 78 ++++++++++++++++++++
 7 files changed, 94 insertions(+), 62 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/20f27999/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSLeafQueue.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSLeafQueue.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSLeafQueue.java
index d8b51f7..6723fd9 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSLeafQueue.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSLeafQueue.java
@@ -43,7 +43,6 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ActiveUsersManager;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerAppUtils;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplicationAttempt;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.policies.DominantResourceFairnessPolicy;
 import org.apache.hadoop.yarn.util.resource.Resources;
 
 @Private
@@ -482,8 +481,7 @@ public class FSLeafQueue extends FSQueue {
 
   /**
    * Check whether this queue can run this application master under the
-   * maxAMShare limit. For FIFO and FAIR policies, check if the VCore usage
-   * takes up the entire cluster or maxResources for the queue.
+   * maxAMShare limit.
    * @param amResource
    * @return true if this queue can run
    */
@@ -493,24 +491,25 @@ public class FSLeafQueue extends FSQueue {
     if (Math.abs(maxAMShare - -1.0f) < 0.0001) {
       return true;
     }
-    Resource maxAMResource = Resources.multiply(getFairShare(), maxAMShare);
-    Resource ifRunAMResource = Resources.add(amResourceUsage, amResource);
 
-    boolean overMaxAMShareLimit = policy
-            .checkIfAMResourceUsageOverLimit(ifRunAMResource, maxAMResource);
+    // If FairShare is zero, use min(maxShare, available resource) to compute
+    // maxAMResource
+    Resource maxResource = Resources.clone(getFairShare());
+    if (maxResource.getMemorySize() == 0) {
+      maxResource.setMemory(
+          Math.min(scheduler.getRootQueueMetrics().getAvailableMB(),
+                   getMaxShare().getMemorySize()));
+    }
 
-    // For fair policy and fifo policy which doesn't check VCore usages,
-    // additionally check if the AM takes all available VCores or
-    // over maxResource to avoid deadlock.
-    if (!overMaxAMShareLimit && !policy.equals(
-        SchedulingPolicy.getInstance(DominantResourceFairnessPolicy.class))) {
-      overMaxAMShareLimit =
-         isVCoresOverMaxResource(ifRunAMResource.getVirtualCores()) ||
-         ifRunAMResource.getVirtualCores() >=
-         scheduler.getRootQueueMetrics().getAvailableVirtualCores();
+    if (maxResource.getVirtualCoresSize() == 0) {
+      maxResource.setVirtualCores(Math.min(
+          scheduler.getRootQueueMetrics().getAvailableVirtualCores(),
+          getMaxShare().getVirtualCoresSize()));
     }
 
-    return !overMaxAMShareLimit;
+    Resource maxAMResource = Resources.multiply(maxResource, maxAMShare);
+    Resource ifRunAMResource = Resources.add(amResourceUsage, amResource);
+    return Resources.fitsIn(ifRunAMResource, maxAMResource);
   }
 
   public void addAMResourceUsage(Resource amResource) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/20f27999/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSQueue.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSQueue.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSQueue.java
index f50c358..25554dd 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSQueue.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSQueue.java
@@ -311,25 +311,6 @@ public abstract class FSQueue implements Queue, Schedulable {
   }
 
   /**
-   * Helper method to check if requested VCores are over maxResource.
-   * @param requestedVCores the number of VCores requested
-   * @return true if the number of VCores requested is over the maxResource;
-   *         false otherwise
-   */
-  protected boolean isVCoresOverMaxResource(int requestedVCores) {
-    if (requestedVCores >= scheduler.getAllocationConfiguration().
-        getMaxResources(getName()).getVirtualCores()) {
-      return true;
-    }
-
-    if (getParent() == null) {
-      return false;
-    }
-
-    return getParent().isVCoresOverMaxResource(requestedVCores);
-  }
-
-  /**
    * Returns true if queue has at least one app running.
    */
   public boolean isActive() {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/20f27999/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/SchedulingPolicy.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/SchedulingPolicy.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/SchedulingPolicy.java
index 160ba4b..9eda46c 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/SchedulingPolicy.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/SchedulingPolicy.java
@@ -178,17 +178,6 @@ public abstract class SchedulingPolicy {
       Resource usage, Resource fairShare);
 
   /**
-   * Check if a leaf queue's AM resource usage over its limit under this policy
-   *
-   * @param usage {@link Resource} the resource used by application masters
-   * @param maxAMResource {@link Resource} the maximum allowed resource for
-   *                                      application masters
-   * @return true if AM resource usage is over the limit
-   */
-  public abstract boolean checkIfAMResourceUsageOverLimit(
-      Resource usage, Resource maxAMResource);
-
-  /**
    * Get headroom by calculating the min of <code>clusterAvailable</code> and
    * (<code>queueFairShare</code> - <code>queueUsage</code>) resources that are
    * applicable to this policy. For eg if only memory then leave other

http://git-wip-us.apache.org/repos/asf/hadoop/blob/20f27999/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/policies/DominantResourceFairnessPolicy.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/policies/DominantResourceFairnessPolicy.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/policies/DominantResourceFairnessPolicy.java
index 623437a..ad41b11 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/policies/DominantResourceFairnessPolicy.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/policies/DominantResourceFairnessPolicy.java
@@ -94,11 +94,6 @@ public class DominantResourceFairnessPolicy extends SchedulingPolicy {
   }
 
   @Override
-  public boolean checkIfAMResourceUsageOverLimit(Resource usage, Resource maxAMResource) {
-    return !Resources.fitsIn(usage, maxAMResource);
-  }
-
-  @Override
   public Resource getHeadroom(Resource queueFairShare, Resource queueUsage,
                               Resource maxAvailable) {
     long queueAvailableMemory =

http://git-wip-us.apache.org/repos/asf/hadoop/blob/20f27999/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/policies/FairSharePolicy.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/policies/FairSharePolicy.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/policies/FairSharePolicy.java
index 42d0420..6aa8405 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/policies/FairSharePolicy.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/policies/FairSharePolicy.java
@@ -151,11 +151,6 @@ public class FairSharePolicy extends SchedulingPolicy {
   }
 
   @Override
-  public boolean checkIfAMResourceUsageOverLimit(Resource usage, Resource maxAMResource) {
-    return usage.getMemorySize() > maxAMResource.getMemorySize();
-  }
-
-  @Override
   public byte getApplicableDepth() {
     return SchedulingPolicy.DEPTH_ANY;
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/20f27999/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/policies/FifoPolicy.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/policies/FifoPolicy.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/policies/FifoPolicy.java
index c277df1..d3fdcf6 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/policies/FifoPolicy.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/policies/FifoPolicy.java
@@ -114,11 +114,6 @@ public class FifoPolicy extends SchedulingPolicy {
   }
 
   @Override
-  public boolean checkIfAMResourceUsageOverLimit(Resource usage, Resource maxAMResource) {
-    return usage.getMemorySize() > maxAMResource.getMemorySize();
-  }
-
-  @Override
   public Resource getHeadroom(Resource queueFairShare,
                               Resource queueUsage, Resource maxAvailable) {
     long queueAvailableMemory = Math.max(

http://git-wip-us.apache.org/repos/asf/hadoop/blob/20f27999/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFairScheduler.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFairScheduler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFairScheduler.java
index 9f22e50..43ebe53 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFairScheduler.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFairScheduler.java
@@ -1081,6 +1081,84 @@ public class TestFairScheduler extends FairSchedulerTestBase {
         getCurrentReservation().getMemorySize());
   }
 
+  /**
+   * The test verifies that zero-FairShare queues (because of zero/tiny
+   * weight) can get resources for the AM.
+   */
+  @Test
+  public void testRequestAMResourceInZeroFairShareQueue() throws Exception {
+    conf.set(FairSchedulerConfiguration.ALLOCATION_FILE, ALLOC_FILE);
+
+    PrintWriter out = new PrintWriter(new FileWriter(ALLOC_FILE));
+    out.println("<?xml version=\"1.0\"?>");
+    out.println("<allocations>");
+    out.println("<queue name=\"queue1\">");
+    out.println("<weight>0.0</weight>");
+    out.println("<maxResources>4096mb,10vcores</maxResources>");
+    out.println("<maxAMShare>0.5</maxAMShare>");
+    out.println("</queue>");
+    out.println("<queue name=\"queue2\">");
+    out.println("<weight>2.0</weight>");
+    out.println("</queue>");
+    out.println("<queue name=\"queue3\">");
+    out.println("<weight>0.000001</weight>");
+    out.println("</queue>");
+    out.println("</allocations>");
+    out.close();
+
+    scheduler.init(conf);
+    scheduler.start();
+    scheduler.reinitialize(conf, resourceManager.getRMContext());
+
+    RMNode node =
+            MockNodes.newNodeInfo(1, Resources.createResource(8192, 20),
+                    0, "127.0.0.1");
+    NodeAddedSchedulerEvent nodeEvent = new NodeAddedSchedulerEvent(node);
+    NodeUpdateSchedulerEvent updateEvent = new NodeUpdateSchedulerEvent(node);
+    scheduler.handle(nodeEvent);
+    scheduler.update();
+
+    //create request for non-zero weight queue
+    createSchedulingRequest(1024, "root.queue2", "user2", 1);
+    scheduler.update();
+    scheduler.handle(updateEvent);
+
+    // A managed AM which need 3G memory will not get resource,
+    // since it request more than the maxAMShare (4G * 0.5 = 2G).
+    Resource amResource = Resource.newInstance(1024, 1);
+    int amPriority = RMAppAttemptImpl.AM_CONTAINER_PRIORITY.getPriority();
+    ApplicationAttemptId attId1 = createAppAttemptId(1, 1);
+    createApplicationWithAMResource(attId1, "root.queue1", "user1", amResource);
+    createSchedulingRequestExistingApplication(3 * 1024, 1, amPriority, attId1);
+    FSAppAttempt app1 = scheduler.getSchedulerApp(attId1);
+    scheduler.update();
+    scheduler.handle(updateEvent);
+    assertEquals("Application 1 should not be running",
+            0, app1.getLiveContainers().size());
+
+    // A managed AM which need 2G memory will get resource,
+    // since it request no more than the maxAMShare (4G * 0.5 = 2G).
+    ApplicationAttemptId attId2 = createAppAttemptId(2, 1);
+    createApplicationWithAMResource(attId2, "root.queue1", "user1", amResource);
+    createSchedulingRequestExistingApplication(2 * 1024, 1, amPriority, attId2);
+    FSAppAttempt app2 = scheduler.getSchedulerApp(attId2);
+    scheduler.update();
+    scheduler.handle(updateEvent);
+    assertEquals("Application 2 should be running",
+            1, app2.getLiveContainers().size());
+
+    // A managed AM which need 1G memory will get resource, even thought its
+    // fair share is 0 because its weight is tiny(0.000001).
+    ApplicationAttemptId attId3 = createAppAttemptId(3, 1);
+    createApplicationWithAMResource(attId3, "root.queue3", "user1", amResource);
+    createSchedulingRequestExistingApplication(1024, 1, amPriority, attId3);
+    FSAppAttempt app3 = scheduler.getSchedulerApp(attId3);
+    scheduler.update();
+    scheduler.handle(updateEvent);
+    assertEquals("Application 3 should be running",
+            1, app3.getLiveContainers().size());
+  }
+
     @Test (timeout = 500000)
   public void testContainerReservationNotExceedingQueueMax() throws Exception {
     conf.set(FairSchedulerConfiguration.ALLOCATION_FILE, ALLOC_FILE);


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


[15/39] hadoop git commit: YARN-5237. Fix missing log files issue in rolling log aggregation. Contributed by Xuan Gong.

Posted by ae...@apache.org.
YARN-5237. Fix missing log files issue in rolling log aggregation. Contributed by Xuan Gong.


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

Branch: refs/heads/HDFS-1312
Commit: 5dfc38ff57669cba9078146e91ed990a1d25a3f0
Parents: 6f0aa75
Author: Junping Du <ju...@apache.org>
Authored: Wed Jun 15 16:17:54 2016 -0700
Committer: Junping Du <ju...@apache.org>
Committed: Wed Jun 15 16:17:54 2016 -0700

----------------------------------------------------------------------
 .../logaggregation/AggregatedLogFormat.java     |  48 +++++----
 .../logaggregation/AppLogAggregatorImpl.java    |   8 +-
 .../TestLogAggregationService.java              | 100 ++++++++++++++++++-
 3 files changed, 134 insertions(+), 22 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/5dfc38ff/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/AggregatedLogFormat.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/AggregatedLogFormat.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/AggregatedLogFormat.java
index 8b213d5..2fb18ed 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/AggregatedLogFormat.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/AggregatedLogFormat.java
@@ -168,6 +168,7 @@ public class AggregatedLogFormat {
     private final Set<String> alreadyUploadedLogFiles;
     private Set<String> allExistingFileMeta = new HashSet<String>();
     private final boolean appFinished;
+    private final boolean containerFinished;
 
     /**
      * The retention context to determine if log files are older than
@@ -186,13 +187,14 @@ public class AggregatedLogFormat {
     public LogValue(List<String> rootLogDirs, ContainerId containerId,
         String user) {
       this(rootLogDirs, containerId, user, null, new HashSet<String>(),
-          null, true);
+          null, true, true);
     }
 
     public LogValue(List<String> rootLogDirs, ContainerId containerId,
         String user, LogAggregationContext logAggregationContext,
         Set<String> alreadyUploadedLogFiles,
-        LogRetentionContext retentionContext, boolean appFinished) {
+        LogRetentionContext retentionContext, boolean appFinished,
+        boolean containerFinished) {
       this.rootLogDirs = new ArrayList<String>(rootLogDirs);
       this.containerId = containerId;
       this.user = user;
@@ -202,6 +204,7 @@ public class AggregatedLogFormat {
       this.logAggregationContext = logAggregationContext;
       this.alreadyUploadedLogFiles = alreadyUploadedLogFiles;
       this.appFinished = appFinished;
+      this.containerFinished = containerFinished;
       this.logRetentionContext = retentionContext;
     }
 
@@ -318,29 +321,40 @@ public class AggregatedLogFormat {
         return candidates;
       }
 
+      Set<File> fileCandidates = new HashSet<File>(candidates);
       if (this.logAggregationContext != null && candidates.size() > 0) {
-        filterFiles(
-          this.appFinished ? this.logAggregationContext.getIncludePattern()
+        fileCandidates = getFileCandidates(fileCandidates, this.appFinished);
+        if (!this.appFinished && this.containerFinished) {
+          Set<File> addition = new HashSet<File>(candidates);
+          addition = getFileCandidates(addition, true);
+          fileCandidates.addAll(addition);
+        }
+      }
+
+      return fileCandidates;
+    }
+
+    private Set<File> getFileCandidates(Set<File> candidates,
+        boolean useRegularPattern) {
+      filterFiles(
+          useRegularPattern ? this.logAggregationContext.getIncludePattern()
               : this.logAggregationContext.getRolledLogsIncludePattern(),
           candidates, false);
 
-        filterFiles(
-          this.appFinished ? this.logAggregationContext.getExcludePattern()
+      filterFiles(
+          useRegularPattern ? this.logAggregationContext.getExcludePattern()
               : this.logAggregationContext.getRolledLogsExcludePattern(),
           candidates, true);
 
-        Iterable<File> mask =
-            Iterables.filter(candidates, new Predicate<File>() {
-              @Override
-              public boolean apply(File next) {
-                return !alreadyUploadedLogFiles
+      Iterable<File> mask =
+          Iterables.filter(candidates, new Predicate<File>() {
+            @Override
+            public boolean apply(File next) {
+              return !alreadyUploadedLogFiles
                   .contains(getLogFileMetaData(next));
-              }
-            });
-        candidates = Sets.newHashSet(mask);
-      }
-
-      return candidates;
+            }
+          });
+      return Sets.newHashSet(mask);
     }
 
     private void filterFiles(String pattern, Set<File> candidates,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5dfc38ff/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/logaggregation/AppLogAggregatorImpl.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/containermanager/logaggregation/AppLogAggregatorImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/logaggregation/AppLogAggregatorImpl.java
index c70fa5b..872b805 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/logaggregation/AppLogAggregatorImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/logaggregation/AppLogAggregatorImpl.java
@@ -328,7 +328,8 @@ public class AppLogAggregatorImpl implements AppLogAggregator {
           containerLogAggregators.put(container, aggregator);
         }
         Set<Path> uploadedFilePathsInThisCycle =
-            aggregator.doContainerLogAggregation(writer, appFinished);
+            aggregator.doContainerLogAggregation(writer, appFinished,
+            finishedContainers.contains(container));
         if (uploadedFilePathsInThisCycle.size() > 0) {
           uploadedLogsInThisCycle = true;
           this.delService.delete(this.userUgi.getShortUserName(), null,
@@ -643,7 +644,7 @@ public class AppLogAggregatorImpl implements AppLogAggregator {
     }
 
     public Set<Path> doContainerLogAggregation(LogWriter writer,
-        boolean appFinished) {
+        boolean appFinished, boolean containerFinished) {
       LOG.info("Uploading logs for container " + containerId
           + ". Current good log dirs are "
           + StringUtils.join(",", dirsHandler.getLogDirsForRead()));
@@ -651,7 +652,8 @@ public class AppLogAggregatorImpl implements AppLogAggregator {
       final LogValue logValue =
           new LogValue(dirsHandler.getLogDirsForRead(), containerId,
             userUgi.getShortUserName(), logAggregationContext,
-            this.uploadedFileMeta,  retentionContext, appFinished);
+            this.uploadedFileMeta,  retentionContext, appFinished,
+            containerFinished);
       try {
         writer.append(logKey, logValue);
       } catch (Exception e) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5dfc38ff/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/logaggregation/TestLogAggregationService.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/logaggregation/TestLogAggregationService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/logaggregation/TestLogAggregationService.java
index 92c6b80..c98e366 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/logaggregation/TestLogAggregationService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/logaggregation/TestLogAggregationService.java
@@ -1575,6 +1575,102 @@ public class TestLogAggregationService extends BaseContainerManagerTest {
       "getApplicationID");
   }
 
+  @SuppressWarnings("resource")
+  @Test (timeout = 50000)
+  public void testLogAggregationServiceWithPatternsAndIntervals()
+      throws Exception {
+    LogAggregationContext logAggregationContext =
+        Records.newRecord(LogAggregationContext.class);
+    // set IncludePattern and RolledLogsIncludePattern.
+    // When the app is running, we only aggregate the log with
+    // the name stdout. After the app finishes, we only aggregate
+    // the log with the name std_final.
+    logAggregationContext.setRolledLogsIncludePattern("stdout");
+    logAggregationContext.setIncludePattern("std_final");
+    this.conf.set(
+        YarnConfiguration.NM_LOG_DIRS, localLogDir.getAbsolutePath());
+    //configure YarnConfiguration.NM_REMOTE_APP_LOG_DIR to
+    //have fully qualified path
+    this.conf.set(YarnConfiguration.NM_REMOTE_APP_LOG_DIR,
+        this.remoteRootLogDir.toURI().toString());
+    this.conf.setLong(
+        YarnConfiguration.NM_LOG_AGGREGATION_ROLL_MONITORING_INTERVAL_SECONDS,
+        3600);
+
+    this.conf.setLong(YarnConfiguration.DEBUG_NM_DELETE_DELAY_SEC, 3600);
+
+    ApplicationId application =
+        BuilderUtils.newApplicationId(System.currentTimeMillis(), 1);
+    ApplicationAttemptId appAttemptId =
+        BuilderUtils.newApplicationAttemptId(application, 1);
+    ContainerId container = createContainer(appAttemptId, 1,
+        ContainerType.APPLICATION_MASTER);
+
+    ConcurrentMap<ApplicationId, Application> maps =
+        this.context.getApplications();
+    Application app = mock(Application.class);
+    maps.put(application, app);
+    when(app.getContainers()).thenReturn(this.context.getContainers());
+
+    LogAggregationService logAggregationService =
+        new LogAggregationService(dispatcher, context, this.delSrvc,
+          super.dirsHandler);
+
+    logAggregationService.init(this.conf);
+    logAggregationService.start();
+
+    // AppLogDir should be created
+    File appLogDir =
+        new File(localLogDir, ConverterUtils.toString(application));
+    appLogDir.mkdir();
+    logAggregationService.handle(new LogHandlerAppStartedEvent(application,
+        this.user, null, this.acls, logAggregationContext));
+
+    // Simulate log-file creation
+    // create std_final in log directory which will not be aggregated
+    // until the app finishes.
+    String[] logFilesWithFinalLog =
+        new String[] {"stdout", "std_final"};
+    writeContainerLogs(appLogDir, container, logFilesWithFinalLog);
+
+    // Do log aggregation
+    AppLogAggregatorImpl aggregator =
+        (AppLogAggregatorImpl) logAggregationService.getAppLogAggregators()
+        .get(application);
+
+    aggregator.doLogAggregationOutOfBand();
+
+    Assert.assertTrue(waitAndCheckLogNum(logAggregationService, application,
+        50, 1, false, null));
+
+    String[] logFiles = new String[] { "stdout" };
+    verifyContainerLogs(logAggregationService, application,
+        new ContainerId[] {container}, logFiles, 1, true);
+
+    logAggregationService.handle(
+        new LogHandlerContainerFinishedEvent(container, 0));
+
+    dispatcher.await();
+
+    // Do the log aggregation after ContainerFinishedEvent but before
+    // AppFinishedEvent. The std_final is expected to be aggregated this time
+    // even if the app is running but the container finishes.
+    aggregator.doLogAggregationOutOfBand();
+
+    Assert.assertTrue(waitAndCheckLogNum(logAggregationService, application,
+        50, 2, false, null));
+
+    // This container finishes.
+    // The log "std_final" should be aggregated this time.
+    String[] logFinalLog = new String[] {"std_final"};
+    verifyContainerLogs(logAggregationService, application,
+        new ContainerId[] {container}, logFinalLog, 1, true);
+
+    logAggregationService.handle(new LogHandlerAppFinishedEvent(application));
+
+    logAggregationService.stop();
+  }
+
   @Test (timeout = 50000)
   @SuppressWarnings("unchecked")
   public void testNoneContainerPolicy() throws Exception {
@@ -1583,14 +1679,14 @@ public class TestLogAggregationService extends BaseContainerManagerTest {
     LogAggregationService logAggregationService = createLogAggregationService(
         appId, NoneContainerLogAggregationPolicy.class, null);
 
-    String[] logFiles = new String[] { "stdout" };
+    String[] logFiles = new String[] {"stdout"};
     ContainerId container1 = finishContainer(appId, logAggregationService,
         ContainerType.APPLICATION_MASTER, 1, 0, logFiles);
 
     finishApplication(appId, logAggregationService);
 
     verifyContainerLogs(logAggregationService, appId,
-        new ContainerId[] { container1 }, logFiles, 0, false);
+        new ContainerId[] {container1}, logFiles, 0, false);
 
     verifyLogAggFinishEvent(appId);
   }


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


[08/39] hadoop git commit: YARN-1942. Deprecate toString/fromString methods from ConverterUtils and move them to records classes like ContainerId/ApplicationId, etc. (wangda)

Posted by ae...@apache.org.
YARN-1942. Deprecate toString/fromString methods from ConverterUtils and move them to records classes like ContainerId/ApplicationId, etc. (wangda)


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

Branch: refs/heads/HDFS-1312
Commit: c77a1095dc556e8bea87df6d8ddf36e898f27e86
Parents: 8e8cb4c
Author: Wangda Tan <wa...@apache.org>
Authored: Tue Jun 14 15:06:38 2016 -0700
Committer: Wangda Tan <wa...@apache.org>
Committed: Tue Jun 14 15:06:38 2016 -0700

----------------------------------------------------------------------
 .../org/apache/hadoop/mapred/YarnChild.java     |  10 +-
 .../hadoop/mapreduce/v2/app/MRAppMaster.java    |   2 +-
 .../v2/app/job/impl/TaskAttemptImpl.java        |   7 +-
 .../v2/app/webapp/dao/TaskAttemptInfo.java      |   4 +-
 .../apache/hadoop/mapreduce/v2/app/MRApp.java   |   2 +-
 .../mapreduce/v2/app/TestMRAppMaster.java       |  66 +++---
 .../app/commit/TestCommitterEventHandler.java   |  12 +-
 .../mapreduce/v2/app/job/impl/TestJobImpl.java  |   4 +-
 .../app/webapp/TestAMWebServicesAttempts.java   |   2 +-
 .../mapred/LocalDistributedCacheManager.java    |   2 +-
 .../apache/hadoop/mapreduce/v2/util/MRApps.java |   8 +-
 .../mapreduce/jobhistory/AMStartedEvent.java    |   6 +-
 .../jobhistory/TaskAttemptStartedEvent.java     |   4 +-
 .../v2/hs/webapp/TestHsWebServicesAttempts.java |   2 +-
 .../org/apache/hadoop/mapred/YARNRunner.java    |   5 +-
 .../apache/hadoop/mapreduce/v2/TestMRJobs.java  |   2 +-
 .../apache/hadoop/mapred/ShuffleHandler.java    |   2 +-
 .../apache/hadoop/tools/HadoopArchiveLogs.java  |   3 +-
 .../yarn/api/records/ApplicationAttemptId.java  |  36 +++-
 .../hadoop/yarn/api/records/ApplicationId.java  |  38 +++-
 .../hadoop/yarn/api/records/ContainerId.java    |   6 +-
 .../apache/hadoop/yarn/api/records/NodeId.java  |  23 ++-
 .../org/apache/hadoop/yarn/api/records/URL.java |  49 +++++
 .../distributedshell/ApplicationMaster.java     |   7 +-
 .../applications/distributedshell/Client.java   |   3 +-
 .../DistributedShellTimelinePlugin.java         |   4 +-
 .../distributedshell/TestDistributedShell.java  |   4 +-
 .../hadoop/yarn/client/cli/ApplicationCLI.java  |  27 ++-
 .../apache/hadoop/yarn/client/cli/LogsCLI.java  |   6 +-
 .../apache/hadoop/yarn/client/cli/NodeCLI.java  |   2 +-
 .../hadoop/yarn/client/cli/RMAdminCLI.java      |   2 +-
 .../hadoop/yarn/client/cli/TestRMAdminCLI.java  |   2 +-
 .../AggregatedLogDeletionService.java           |   2 +-
 .../logaggregation/AggregatedLogFormat.java     |  11 +-
 .../yarn/logaggregation/LogCLIHelpers.java      |   2 +-
 .../apache/hadoop/yarn/util/ConverterUtils.java | 206 +++++++------------
 .../org/apache/hadoop/yarn/util/FSDownload.java |   2 +-
 .../yarn/webapp/log/AggregatedLogsBlock.java    |   4 +-
 .../hadoop/yarn/webapp/util/WebAppUtils.java    |   2 +-
 .../hadoop/yarn/util/TestConverterUtils.java    |  30 +--
 .../apache/hadoop/yarn/util/TestFSDownload.java |  13 +-
 ...pplicationHistoryManagerOnTimelineStore.java |  25 +--
 .../FileSystemApplicationHistoryStore.java      |   8 +-
 .../hadoop/yarn/server/utils/BuilderUtils.java  |   2 +-
 .../yarn/server/webapp/AppAttemptBlock.java     |   2 +-
 .../yarn/server/webapp/ContainerBlock.java      |   2 +-
 .../hadoop/yarn/server/webapp/WebServices.java  |   6 +-
 .../nodemanager/DefaultContainerExecutor.java   |   5 +-
 .../nodemanager/DockerContainerExecutor.java    |   6 +-
 .../nodemanager/LinuxContainerExecutor.java     |   2 +-
 .../container/ContainerImpl.java                |   2 +-
 .../launcher/ContainerLaunch.java               |  10 +-
 .../launcher/ContainerRelaunch.java             |   2 +-
 .../launcher/RecoveredContainerLaunch.java      |   6 +-
 .../localizer/ContainerLocalizer.java           |   3 +-
 .../localizer/LocalResourceRequest.java         |   4 +-
 .../localizer/ResourceLocalizationService.java  |  28 +--
 .../event/LocalizerResourceRequestEvent.java    |   2 +-
 .../sharedcache/SharedCacheUploader.java        |   2 +-
 .../logaggregation/AppLogAggregatorImpl.java    |   2 +-
 .../recovery/NMLeveldbStateStoreService.java    |  10 +-
 .../util/NodeManagerBuilderUtils.java           |   2 +-
 .../nodemanager/util/ProcessIdFileReader.java   |   3 +-
 .../nodemanager/webapp/ApplicationPage.java     |   5 +-
 .../nodemanager/webapp/ContainerLogsPage.java   |   2 +-
 .../nodemanager/webapp/ContainerLogsUtils.java  |   6 +-
 .../nodemanager/webapp/ContainerPage.java       |   2 +-
 .../nodemanager/webapp/NMWebServices.java       |   4 +-
 .../server/nodemanager/webapp/dao/AppInfo.java  |   4 +-
 .../nodemanager/TestNodeManagerReboot.java      |   2 +-
 .../nodemanager/TestNodeManagerResync.java      |   2 +-
 .../nodemanager/TestNodeManagerShutdown.java    |   2 +-
 .../impl/pb/TestPBRecordImpl.java               |   9 +-
 .../containermanager/TestContainerManager.java  |  22 +-
 .../TestContainerManagerRecovery.java           |   2 +-
 .../launcher/TestContainerLaunch.java           |   8 +-
 .../localizer/TestContainerLocalizer.java       |   2 +-
 .../localizer/TestLocalResource.java            |   7 +-
 .../TestResourceLocalizationService.java        |  21 +-
 .../TestAppLogAggregatorImpl.java               |  14 +-
 .../TestLogAggregationService.java              |  36 ++--
 .../monitor/TestContainersMonitor.java          |   2 +-
 .../TestNMLeveldbStateStoreService.java         |  24 +--
 .../nodemanager/webapp/TestNMWebServer.java     |   2 +-
 .../webapp/TestNMWebServicesContainers.java     |   7 +-
 .../server/resourcemanager/ResourceManager.java |   2 +-
 .../recovery/LeveldbRMStateStore.java           |   5 +-
 .../recovery/ZKRMStateStore.java                |   2 +-
 .../resource/DynamicResourceConfiguration.java  |   2 +-
 .../rmcontainer/RMContainerImpl.java            |   2 +-
 .../resourcemanager/webapp/RMAppsBlock.java     |   4 +-
 .../resourcemanager/webapp/RMWebAppFilter.java  |   2 +-
 .../resourcemanager/webapp/RMWebServices.java   |   9 +-
 .../webapp/dao/AppAttemptInfo.java              |   2 +-
 .../resourcemanager/webapp/dao/AppInfo.java     |   3 +-
 .../resourcemanager/TestRMAdminService.java     |   8 +-
 .../recovery/RMStateStoreTestBase.java          |  14 +-
 .../recovery/TestFSRMStateStore.java            |   4 +-
 .../recovery/TestZKRMStateStore.java            |  11 +-
 .../TestRMWebServicesAppsModification.java      |   7 +-
 ...ebServicesDelegationTokenAuthentication.java |   5 +-
 .../webapp/TestRMWebappAuthentication.java      |   9 +-
 .../timeline/EntityGroupFSTimelineStore.java    |   2 +-
 .../timeline/EntityGroupPlugInForTest.java      |   7 +-
 .../TestEntityGroupFSTimelineStore.java         |   4 +-
 105 files changed, 561 insertions(+), 486 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/c77a1095/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapred/YarnChild.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapred/YarnChild.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapred/YarnChild.java
index ec7ade7..164f19d 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapred/YarnChild.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapred/YarnChild.java
@@ -58,6 +58,7 @@ import org.apache.hadoop.yarn.YarnUncaughtExceptionHandler;
 import org.apache.hadoop.yarn.api.ApplicationConstants;
 import org.apache.hadoop.yarn.api.ApplicationConstants.Environment;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
+import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.util.ConverterUtils;
 
 /**
@@ -290,11 +291,10 @@ class YarnChild {
   private static void configureTask(JobConf job, Task task,
       Credentials credentials, Token<JobTokenIdentifier> jt) throws IOException {
     job.setCredentials(credentials);
-    
-    ApplicationAttemptId appAttemptId =
-        ConverterUtils.toContainerId(
-            System.getenv(Environment.CONTAINER_ID.name()))
-            .getApplicationAttemptId();
+
+    ApplicationAttemptId appAttemptId = ContainerId.fromString(
+        System.getenv(Environment.CONTAINER_ID.name()))
+        .getApplicationAttemptId();
     LOG.debug("APPLICATION_ATTEMPT_ID: " + appAttemptId);
     // Set it in conf, so as to be able to be used the the OutputCommitter.
     job.setInt(MRJobConfig.APPLICATION_ATTEMPT_ID,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c77a1095/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/MRAppMaster.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/MRAppMaster.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/MRAppMaster.java
index f8d54c5..c5070f3 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/MRAppMaster.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/MRAppMaster.java
@@ -1562,7 +1562,7 @@ public class MRAppMaster extends CompositeService {
       validateInputParam(appSubmitTimeStr,
           ApplicationConstants.APP_SUBMIT_TIME_ENV);
 
-      ContainerId containerId = ConverterUtils.toContainerId(containerIdStr);
+      ContainerId containerId = ContainerId.fromString(containerIdStr);
       ApplicationAttemptId applicationAttemptId =
           containerId.getApplicationAttemptId();
       if (applicationAttemptId != null) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c77a1095/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TaskAttemptImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TaskAttemptImpl.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TaskAttemptImpl.java
index 6ee8e00..0da3afb 100755
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TaskAttemptImpl.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TaskAttemptImpl.java
@@ -713,8 +713,7 @@ public abstract class TaskAttemptImpl implements
       LocalResourceType type, LocalResourceVisibility visibility)
       throws IOException {
     FileStatus fstat = fc.getFileStatus(file);
-    URL resourceURL = ConverterUtils.getYarnUrlFromPath(fc.resolvePath(fstat
-        .getPath()));
+    URL resourceURL = URL.fromPath(fc.resolvePath(fstat.getPath()));
     long resourceSize = fstat.getLen();
     long resourceModificationTime = fstat.getModificationTime();
 
@@ -1247,8 +1246,8 @@ public abstract class TaskAttemptImpl implements
   public TaskAttemptStateInternal recover(TaskAttemptInfo taInfo,
       OutputCommitter committer, boolean recoverOutput) {
     ContainerId containerId = taInfo.getContainerId();
-    NodeId containerNodeId = ConverterUtils.toNodeId(taInfo.getHostname() + ":"
-        + taInfo.getPort());
+    NodeId containerNodeId = NodeId.fromString(
+        taInfo.getHostname() + ":" + taInfo.getPort());
     String nodeHttpAddress = StringInterner.weakIntern(taInfo.getHostname() + ":"
         + taInfo.getHttpPort());
     // Resource/Priority/Tokens are only needed while launching the container on

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c77a1095/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/dao/TaskAttemptInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/dao/TaskAttemptInfo.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/dao/TaskAttemptInfo.java
index d8e89b1..892c626 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/dao/TaskAttemptInfo.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/dao/TaskAttemptInfo.java
@@ -69,8 +69,10 @@ public class TaskAttemptInfo {
     this.nodeHttpAddress = ta.getNodeHttpAddress();
     this.startTime = report.getStartTime();
     this.finishTime = report.getFinishTime();
-    this.assignedContainerId = ConverterUtils.toString(report.getContainerId());
     this.assignedContainer = report.getContainerId();
+    if (assignedContainer != null) {
+      this.assignedContainerId = assignedContainer.toString();
+    }
     this.progress = report.getProgress() * 100;
     this.status = report.getStateString();
     this.state = report.getTaskAttemptState();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c77a1095/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/MRApp.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/MRApp.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/MRApp.java
index b43a7b4..6ba93e6 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/MRApp.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/MRApp.java
@@ -254,7 +254,7 @@ public class MRApp extends MRAppMaster {
     // the job can reaches the final state when MRAppMaster shuts down.
     this.successfullyUnregistered.set(unregistered);
     this.assignedQueue = assignedQueue;
-    this.resource = Resource.newInstance(1234, 2);
+    this.resource = Resource.newInstance(1234L, 2L);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c77a1095/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestMRAppMaster.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestMRAppMaster.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestMRAppMaster.java
index 5116491..203958d 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestMRAppMaster.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestMRAppMaster.java
@@ -114,7 +114,7 @@ public class TestMRAppMaster {
     localFS.delete(testDir, true);
     new File(testDir.toString()).mkdir();
   }
-  
+
   @Before
   public void prepare() throws IOException {
     File dir = new File(stagingDir);
@@ -134,11 +134,11 @@ public class TestMRAppMaster {
       InterruptedException {
     String applicationAttemptIdStr = "appattempt_1317529182569_0004_000001";
     String containerIdStr = "container_1317529182569_0004_000001_1";
-    
+
     String userName = "TestAppMasterUser";
-    ApplicationAttemptId applicationAttemptId = ConverterUtils
-        .toApplicationAttemptId(applicationAttemptIdStr);
-    ContainerId containerId = ConverterUtils.toContainerId(containerIdStr);
+    ApplicationAttemptId applicationAttemptId = ApplicationAttemptId.fromString(
+        applicationAttemptIdStr);
+    ContainerId containerId = ContainerId.fromString(containerIdStr);
     MRAppMasterTest appMaster =
         new MRAppMasterTest(applicationAttemptId, containerId, "host", -1, -1,
             System.currentTimeMillis());
@@ -161,15 +161,15 @@ public class TestMRAppMaster {
     conf.set(MRJobConfig.MR_AM_STAGING_DIR, stagingDir);
     conf.setInt(org.apache.hadoop.mapreduce.lib.output.FileOutputCommitter.
         FILEOUTPUTCOMMITTER_ALGORITHM_VERSION, 1);
-    ApplicationAttemptId applicationAttemptId = ConverterUtils
-        .toApplicationAttemptId(applicationAttemptIdStr);
+    ApplicationAttemptId applicationAttemptId = ApplicationAttemptId.fromString(
+        applicationAttemptIdStr);
     JobId jobId =  TypeConverter.toYarn(
         TypeConverter.fromYarn(applicationAttemptId.getApplicationId()));
     Path start = MRApps.getStartJobCommitFile(conf, userName, jobId);
     FileSystem fs = FileSystem.get(conf);
     //Create the file, but no end file so we should unregister with an error.
     fs.create(start).close();
-    ContainerId containerId = ConverterUtils.toContainerId(containerIdStr);
+    ContainerId containerId = ContainerId.fromString(containerIdStr);
     MRAppMaster appMaster =
         new MRAppMasterTest(applicationAttemptId, containerId, "host", -1, -1,
             System.currentTimeMillis(), false, false);
@@ -200,8 +200,8 @@ public class TestMRAppMaster {
     conf.set(MRJobConfig.MR_AM_STAGING_DIR, stagingDir);
     conf.setInt(MRJobConfig.NUM_REDUCES, 0);
     conf.set(JHAdminConfig.MR_HS_JHIST_FORMAT, "json");
-    ApplicationAttemptId applicationAttemptId = ConverterUtils
-        .toApplicationAttemptId(applicationAttemptIdStr);
+    ApplicationAttemptId applicationAttemptId = ApplicationAttemptId.fromString(
+        applicationAttemptIdStr);
     JobId jobId = TypeConverter.toYarn(
         TypeConverter.fromYarn(applicationAttemptId.getApplicationId()));
 
@@ -219,7 +219,7 @@ public class TestMRAppMaster {
     FileSystem fs = FileSystem.get(conf);
     JobSplitWriter.createSplitFiles(new Path(dir.getAbsolutePath()), conf,
         fs, new org.apache.hadoop.mapred.InputSplit[0]);
-    ContainerId containerId = ConverterUtils.toContainerId(containerIdStr);
+    ContainerId containerId = ContainerId.fromString(containerIdStr);
     MRAppMasterTestLaunchTime appMaster =
         new MRAppMasterTestLaunchTime(applicationAttemptId, containerId,
             "host", -1, -1, System.currentTimeMillis());
@@ -237,8 +237,8 @@ public class TestMRAppMaster {
     String userName = "TestAppMasterUser";
     JobConf conf = new JobConf();
     conf.set(MRJobConfig.MR_AM_STAGING_DIR, stagingDir);
-    ApplicationAttemptId applicationAttemptId = ConverterUtils
-        .toApplicationAttemptId(applicationAttemptIdStr);
+    ApplicationAttemptId applicationAttemptId = ApplicationAttemptId.fromString(
+        applicationAttemptIdStr);
     JobId jobId =  TypeConverter.toYarn(
         TypeConverter.fromYarn(applicationAttemptId.getApplicationId()));
     Path start = MRApps.getStartJobCommitFile(conf, userName, jobId);
@@ -246,7 +246,7 @@ public class TestMRAppMaster {
     FileSystem fs = FileSystem.get(conf);
     fs.create(start).close();
     fs.create(end).close();
-    ContainerId containerId = ConverterUtils.toContainerId(containerIdStr);
+    ContainerId containerId = ContainerId.fromString(containerIdStr);
     MRAppMaster appMaster =
         new MRAppMasterTest(applicationAttemptId, containerId, "host", -1, -1,
             System.currentTimeMillis(), false, false);
@@ -266,7 +266,7 @@ public class TestMRAppMaster {
     // verify the final status is SUCCEEDED
     verifyFailedStatus((MRAppMasterTest)appMaster, "SUCCEEDED");
   }
-  
+
   @Test
   public void testMRAppMasterFailLock() throws IOException,
       InterruptedException {
@@ -275,8 +275,8 @@ public class TestMRAppMaster {
     String userName = "TestAppMasterUser";
     JobConf conf = new JobConf();
     conf.set(MRJobConfig.MR_AM_STAGING_DIR, stagingDir);
-    ApplicationAttemptId applicationAttemptId = ConverterUtils
-        .toApplicationAttemptId(applicationAttemptIdStr);
+    ApplicationAttemptId applicationAttemptId = ApplicationAttemptId.fromString(
+        applicationAttemptIdStr);
     JobId jobId =  TypeConverter.toYarn(
         TypeConverter.fromYarn(applicationAttemptId.getApplicationId()));
     Path start = MRApps.getStartJobCommitFile(conf, userName, jobId);
@@ -284,7 +284,7 @@ public class TestMRAppMaster {
     FileSystem fs = FileSystem.get(conf);
     fs.create(start).close();
     fs.create(end).close();
-    ContainerId containerId = ConverterUtils.toContainerId(containerIdStr);
+    ContainerId containerId = ContainerId.fromString(containerIdStr);
     MRAppMaster appMaster =
         new MRAppMasterTest(applicationAttemptId, containerId, "host", -1, -1,
             System.currentTimeMillis(), false, false);
@@ -304,7 +304,7 @@ public class TestMRAppMaster {
     // verify the final status is FAILED
     verifyFailedStatus((MRAppMasterTest)appMaster, "FAILED");
   }
-  
+
   @Test
   public void testMRAppMasterMissingStaging() throws IOException,
       InterruptedException {
@@ -313,16 +313,16 @@ public class TestMRAppMaster {
     String userName = "TestAppMasterUser";
     JobConf conf = new JobConf();
     conf.set(MRJobConfig.MR_AM_STAGING_DIR, stagingDir);
-    ApplicationAttemptId applicationAttemptId = ConverterUtils
-        .toApplicationAttemptId(applicationAttemptIdStr);
+    ApplicationAttemptId applicationAttemptId = ApplicationAttemptId.fromString(
+        applicationAttemptIdStr);
 
     //Delete the staging directory
     File dir = new File(stagingDir);
     if(dir.exists()) {
       FileUtils.deleteDirectory(dir);
     }
-    
-    ContainerId containerId = ConverterUtils.toContainerId(containerIdStr);
+
+    ContainerId containerId = ContainerId.fromString(containerIdStr);
     MRAppMaster appMaster =
         new MRAppMasterTest(applicationAttemptId, containerId, "host", -1, -1,
             System.currentTimeMillis(), false, false);
@@ -353,9 +353,9 @@ public class TestMRAppMaster {
     String containerIdStr = "container_1317529182569_0004_000002_1";
 
     String userName = "TestAppMasterUser";
-    ApplicationAttemptId applicationAttemptId = ConverterUtils
-        .toApplicationAttemptId(applicationAttemptIdStr);
-    ContainerId containerId = ConverterUtils.toContainerId(containerIdStr);
+    ApplicationAttemptId applicationAttemptId = ApplicationAttemptId.fromString(
+        applicationAttemptIdStr);
+    ContainerId containerId = ContainerId.fromString(containerIdStr);
     JobConf conf = new JobConf();
     conf.set(MRJobConfig.MR_AM_STAGING_DIR, stagingDir);
 
@@ -427,7 +427,7 @@ public class TestMRAppMaster {
         new Token<AMRMTokenIdentifier>(identifier, password,
             AMRMTokenIdentifier.KIND_NAME, appTokenService);
     credentials.addToken(appTokenService, appToken);
-    
+
     Text keyAlias = new Text("mySecretKeyAlias");
     credentials.addSecretKey(keyAlias, "mySecretKey".getBytes());
     Token<? extends TokenIdentifier> storedToken =
@@ -488,7 +488,7 @@ public class TestMRAppMaster {
     Assert.assertEquals(storedToken, confCredentials.getToken(tokenAlias));
     Assert.assertEquals("mySecretKey",
       new String(confCredentials.getSecretKey(keyAlias)));
-    
+
     // Verify the AM's ugi - app token should be present
     Credentials ugiCredentials = appMaster.getUgi().getCredentials();
     Assert.assertEquals(1, ugiCredentials.numberOfSecretKeys());
@@ -507,9 +507,9 @@ public class TestMRAppMaster {
     String applicationAttemptIdStr = "appattempt_1317529182569_0004_000002";
     String containerIdStr = "container_1317529182569_0004_000002_1";
     String userName = "TestAppMasterUser";
-    ApplicationAttemptId applicationAttemptId = ConverterUtils
-        .toApplicationAttemptId(applicationAttemptIdStr);
-    ContainerId containerId = ConverterUtils.toContainerId(containerIdStr);
+    ApplicationAttemptId applicationAttemptId = ApplicationAttemptId.fromString(
+        applicationAttemptIdStr);
+    ContainerId containerId = ContainerId.fromString(containerIdStr);
     JobConf conf = new JobConf();
     conf.set(MRJobConfig.MR_AM_STAGING_DIR, stagingDir);
 
@@ -591,7 +591,7 @@ class MRAppMasterTest extends MRAppMaster {
     }
     this.conf = conf;
   }
-  
+
   @Override
   protected ContainerAllocator createContainerAllocator(
       final ClientService clientService, final AppContext context) {
@@ -628,7 +628,7 @@ class MRAppMasterTest extends MRAppMaster {
   public Credentials getCredentials() {
     return super.getCredentials();
   }
-  
+
   public UserGroupInformation getUgi() {
     return currentUser;
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c77a1095/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/commit/TestCommitterEventHandler.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/commit/TestCommitterEventHandler.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/commit/TestCommitterEventHandler.java
index a4853d5..b099bcc 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/commit/TestCommitterEventHandler.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/commit/TestCommitterEventHandler.java
@@ -129,8 +129,8 @@ public class TestCommitterEventHandler {
 
     SystemClock clock = SystemClock.getInstance();
     AppContext appContext = mock(AppContext.class);
-    ApplicationAttemptId attemptid = 
-      ConverterUtils.toApplicationAttemptId("appattempt_1234567890000_0001_0");
+    ApplicationAttemptId attemptid = ApplicationAttemptId.fromString(
+        "appattempt_1234567890000_0001_0");
     when(appContext.getApplicationID()).thenReturn(attemptid.getApplicationId());
     when(appContext.getApplicationAttemptId()).thenReturn(attemptid);
     when(appContext.getEventHandler()).thenReturn(
@@ -240,8 +240,8 @@ public class TestCommitterEventHandler {
     YarnConfiguration conf = new YarnConfiguration();
     conf.set(MRJobConfig.MR_AM_STAGING_DIR, stagingDir);
     JobContext mockJobContext = mock(JobContext.class);
-    ApplicationAttemptId attemptid = 
-      ConverterUtils.toApplicationAttemptId("appattempt_1234567890000_0001_0");
+    ApplicationAttemptId attemptid = ApplicationAttemptId.fromString(
+        "appattempt_1234567890000_0001_0");
     JobId jobId =  TypeConverter.toYarn(
         TypeConverter.fromYarn(attemptid.getApplicationId()));
     
@@ -288,8 +288,8 @@ public class TestCommitterEventHandler {
     YarnConfiguration conf = new YarnConfiguration();
     conf.set(MRJobConfig.MR_AM_STAGING_DIR, stagingDir);
     JobContext mockJobContext = mock(JobContext.class);
-    ApplicationAttemptId attemptid = 
-      ConverterUtils.toApplicationAttemptId("appattempt_1234567890000_0001_0");
+    ApplicationAttemptId attemptid =
+        ApplicationAttemptId.fromString("appattempt_1234567890000_0001_0");
     JobId jobId =  TypeConverter.toYarn(
         TypeConverter.fromYarn(attemptid.getApplicationId()));
     

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c77a1095/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TestJobImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TestJobImpl.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TestJobImpl.java
index 36221e0..eaa5af7 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TestJobImpl.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TestJobImpl.java
@@ -942,8 +942,8 @@ public class TestJobImpl {
         callback.run();
       }
     };
-    ApplicationAttemptId id = 
-      ConverterUtils.toApplicationAttemptId("appattempt_1234567890000_0001_0");
+    ApplicationAttemptId id = ApplicationAttemptId.fromString(
+        "appattempt_1234567890000_0001_0");
     when(appContext.getApplicationID()).thenReturn(id.getApplicationId());
     when(appContext.getApplicationAttemptId()).thenReturn(id);
     CommitterEventHandler handler =

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c77a1095/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/webapp/TestAMWebServicesAttempts.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/webapp/TestAMWebServicesAttempts.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/webapp/TestAMWebServicesAttempts.java
index dcd5d29..3c9127f 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/webapp/TestAMWebServicesAttempts.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/webapp/TestAMWebServicesAttempts.java
@@ -515,7 +515,7 @@ public class TestAMWebServicesAttempts extends JerseyTest {
     WebServicesTestUtils.checkStringMatch("diagnostics", expectDiag,
         diagnostics);
     WebServicesTestUtils.checkStringMatch("assignedContainerId",
-        ConverterUtils.toString(ta.getAssignedContainerID()),
+        ta.getAssignedContainerID().toString(),
         assignedContainerId);
 
     assertEquals("startTime wrong", ta.getLaunchTime(), startTime);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c77a1095/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapred/LocalDistributedCacheManager.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapred/LocalDistributedCacheManager.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapred/LocalDistributedCacheManager.java
index 3b87197..c58a774 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapred/LocalDistributedCacheManager.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapred/LocalDistributedCacheManager.java
@@ -157,7 +157,7 @@ class LocalDistributedCacheManager {
         }
         Path resourcePath;
         try {
-          resourcePath = ConverterUtils.getPathFromYarnURL(resource.getResource());
+          resourcePath = resource.getResource().toPath();
         } catch (URISyntaxException e) {
           throw new IOException(e);
         }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c77a1095/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapreduce/v2/util/MRApps.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapreduce/v2/util/MRApps.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapreduce/v2/util/MRApps.java
index c8d8a44..31e4c0f 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapreduce/v2/util/MRApps.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapreduce/v2/util/MRApps.java
@@ -68,6 +68,7 @@ import org.apache.hadoop.yarn.api.ApplicationConstants.Environment;
 import org.apache.hadoop.yarn.api.records.LocalResource;
 import org.apache.hadoop.yarn.api.records.LocalResourceType;
 import org.apache.hadoop.yarn.api.records.LocalResourceVisibility;
+import org.apache.hadoop.yarn.api.records.URL;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
 import org.apache.hadoop.yarn.util.Apps;
@@ -569,14 +570,13 @@ public class MRApps extends Apps {
         }
         String linkName = name.toUri().getPath();
         LocalResource orig = localResources.get(linkName);
-        if(orig != null && !orig.getResource().equals(
-            ConverterUtils.getYarnUrlFromURI(p.toUri()))) {
+        if(orig != null && !orig.getResource().equals(URL.fromURI(p.toUri()))) {
           throw new InvalidJobConfException(
               getResourceDescription(orig.getType()) + orig.getResource() + 
               " conflicts with " + getResourceDescription(type) + u);
         }
-        localResources.put(linkName, LocalResource.newInstance(ConverterUtils
-          .getYarnUrlFromURI(p.toUri()), type, visibilities[i]
+        localResources.put(linkName, LocalResource
+            .newInstance(URL.fromURI(p.toUri()), type, visibilities[i]
             ? LocalResourceVisibility.PUBLIC : LocalResourceVisibility.PRIVATE,
           sizes[i], timestamps[i]));
       }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c77a1095/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/AMStartedEvent.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/AMStartedEvent.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/AMStartedEvent.java
index ea2ca9e..266aa94 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/AMStartedEvent.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/AMStartedEvent.java
@@ -107,8 +107,8 @@ public class AMStartedEvent implements HistoryEvent {
    * @return the ApplicationAttemptId
    */
   public ApplicationAttemptId getAppAttemptId() {
-    return ConverterUtils.toApplicationAttemptId(datum.getApplicationAttemptId()
-        .toString());
+    return ApplicationAttemptId.fromString(
+        datum.getApplicationAttemptId().toString());
   }
 
   /**
@@ -122,7 +122,7 @@ public class AMStartedEvent implements HistoryEvent {
    * @return the ContainerId for the MRAppMaster.
    */
   public ContainerId getContainerId() {
-    return ConverterUtils.toContainerId(datum.getContainerId().toString());
+    return ContainerId.fromString(datum.getContainerId().toString());
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c77a1095/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/TaskAttemptStartedEvent.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/TaskAttemptStartedEvent.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/TaskAttemptStartedEvent.java
index c8c250a..3073d5b 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/TaskAttemptStartedEvent.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/TaskAttemptStartedEvent.java
@@ -75,7 +75,7 @@ public class TaskAttemptStartedEvent implements HistoryEvent {
       long startTime, String trackerName, int httpPort, int shufflePort,
       String locality, String avataar) {
     this(attemptId, taskType, startTime, trackerName, httpPort, shufflePort,
-        ConverterUtils.toContainerId("container_-1_-1_-1_-1"), locality,
+        ContainerId.fromString("container_-1_-1_-1_-1"), locality,
             avataar);
   }
 
@@ -116,7 +116,7 @@ public class TaskAttemptStartedEvent implements HistoryEvent {
   }
   /** Get the ContainerId */
   public ContainerId getContainerId() {
-    return ConverterUtils.toContainerId(datum.getContainerId().toString());
+    return ContainerId.fromString(datum.getContainerId().toString());
   }
   /** Get the locality */
   public String getLocality() {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c77a1095/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/test/java/org/apache/hadoop/mapreduce/v2/hs/webapp/TestHsWebServicesAttempts.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/test/java/org/apache/hadoop/mapreduce/v2/hs/webapp/TestHsWebServicesAttempts.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/test/java/org/apache/hadoop/mapreduce/v2/hs/webapp/TestHsWebServicesAttempts.java
index 60dc235..54c2792 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/test/java/org/apache/hadoop/mapreduce/v2/hs/webapp/TestHsWebServicesAttempts.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/test/java/org/apache/hadoop/mapreduce/v2/hs/webapp/TestHsWebServicesAttempts.java
@@ -533,7 +533,7 @@ public class TestHsWebServicesAttempts extends JerseyTest {
     WebServicesTestUtils.checkStringMatch("diagnostics", expectDiag,
         diagnostics);
     WebServicesTestUtils.checkStringMatch("assignedContainerId",
-        ConverterUtils.toString(ta.getAssignedContainerID()),
+        ta.getAssignedContainerID().toString(),
         assignedContainerId);
 
     assertEquals("startTime wrong", ta.getLaunchTime(), startTime);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c77a1095/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/main/java/org/apache/hadoop/mapred/YARNRunner.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/main/java/org/apache/hadoop/mapred/YARNRunner.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/main/java/org/apache/hadoop/mapred/YARNRunner.java
index 1342282..b30641e 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/main/java/org/apache/hadoop/mapred/YARNRunner.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/main/java/org/apache/hadoop/mapred/YARNRunner.java
@@ -321,7 +321,7 @@ public class YARNRunner implements ClientProtocol {
       throws IOException {
     LocalResource rsrc = recordFactory.newRecordInstance(LocalResource.class);
     FileStatus rsrcStat = fs.getFileStatus(p);
-    rsrc.setResource(ConverterUtils.getYarnUrlFromPath(fs
+    rsrc.setResource(URL.fromPath(fs
         .getDefaultFileSystem().resolvePath(rsrcStat.getPath())));
     rsrc.setSize(rsrcStat.getLen());
     rsrc.setTimestamp(rsrcStat.getModificationTime());
@@ -355,8 +355,7 @@ public class YARNRunner implements ClientProtocol {
 
     Path jobConfPath = new Path(jobSubmitDir, MRJobConfig.JOB_CONF_FILE);
 
-    URL yarnUrlForJobSubmitDir = ConverterUtils
-        .getYarnUrlFromPath(defaultFileContext.getDefaultFileSystem()
+    URL yarnUrlForJobSubmitDir = URL.fromPath(defaultFileContext.getDefaultFileSystem()
             .resolvePath(
                 defaultFileContext.makeQualified(new Path(jobSubmitDir))));
     LOG.debug("Creating setup context, jobSubmitDir url is "

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c77a1095/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/v2/TestMRJobs.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/v2/TestMRJobs.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/v2/TestMRJobs.java
index a6647f1..900bdeb 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/v2/TestMRJobs.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/v2/TestMRJobs.java
@@ -749,7 +749,7 @@ public class TestMRJobs {
           boolean foundAppMaster = job.isUber();
           final Path containerPathComponent = slog.getPath().getParent();
           if (!foundAppMaster) {
-            final ContainerId cid = ConverterUtils.toContainerId(
+            final ContainerId cid = ContainerId.fromString(
                 containerPathComponent.getName());
             foundAppMaster =
                 ((cid.getContainerId() & ContainerId.CONTAINER_ID_BITMASK)== 1);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c77a1095/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-shuffle/src/main/java/org/apache/hadoop/mapred/ShuffleHandler.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-shuffle/src/main/java/org/apache/hadoop/mapred/ShuffleHandler.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-shuffle/src/main/java/org/apache/hadoop/mapred/ShuffleHandler.java
index 0d6e900..8cbae81 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-shuffle/src/main/java/org/apache/hadoop/mapred/ShuffleHandler.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-shuffle/src/main/java/org/apache/hadoop/mapred/ShuffleHandler.java
@@ -999,7 +999,7 @@ public class ShuffleHandler extends AuxiliaryService {
       final String baseStr =
           ContainerLocalizer.USERCACHE + "/" + user + "/"
               + ContainerLocalizer.APPCACHE + "/"
-              + ConverterUtils.toString(appID) + "/output" + "/";
+              + appID.toString() + "/output" + "/";
       return baseStr;
     }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c77a1095/hadoop-tools/hadoop-archive-logs/src/main/java/org/apache/hadoop/tools/HadoopArchiveLogs.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-archive-logs/src/main/java/org/apache/hadoop/tools/HadoopArchiveLogs.java b/hadoop-tools/hadoop-archive-logs/src/main/java/org/apache/hadoop/tools/HadoopArchiveLogs.java
index c502ffd..2e44070 100644
--- a/hadoop-tools/hadoop-archive-logs/src/main/java/org/apache/hadoop/tools/HadoopArchiveLogs.java
+++ b/hadoop-tools/hadoop-archive-logs/src/main/java/org/apache/hadoop/tools/HadoopArchiveLogs.java
@@ -39,6 +39,7 @@ import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.mapred.JobConf;
 import org.apache.hadoop.util.Tool;
 import org.apache.hadoop.util.ToolRunner;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ApplicationReport;
 import org.apache.hadoop.yarn.api.records.LogAggregationStatus;
 import org.apache.hadoop.yarn.applications.distributedshell.ApplicationMaster;
@@ -302,7 +303,7 @@ public class HadoopArchiveLogs implements Tool {
         AppInfo app = it.next();
         try {
           ApplicationReport report = client.getApplicationReport(
-              ConverterUtils.toApplicationId(app.getAppId()));
+              ApplicationId.fromString(app.getAppId()));
           LogAggregationStatus aggStatus = report.getLogAggregationStatus();
           if (aggStatus.equals(LogAggregationStatus.RUNNING) ||
               aggStatus.equals(LogAggregationStatus.RUNNING_WITH_FAILURE) ||

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c77a1095/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ApplicationAttemptId.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ApplicationAttemptId.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ApplicationAttemptId.java
index 0a83bc0..5f3a68e 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ApplicationAttemptId.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ApplicationAttemptId.java
@@ -19,6 +19,8 @@
 package org.apache.hadoop.yarn.api.records;
 
 import java.text.NumberFormat;
+import java.util.Iterator;
+import java.util.NoSuchElementException;
 
 import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.classification.InterfaceAudience.Public;
@@ -26,6 +28,8 @@ import org.apache.hadoop.classification.InterfaceStability.Stable;
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.yarn.util.Records;
 
+import com.google.common.base.Splitter;
+
 /**
  * <p><code>ApplicationAttemptId</code> denotes the particular <em>attempt</em>
  * of an <code>ApplicationMaster</code> for a given {@link ApplicationId}.</p>
@@ -38,10 +42,11 @@ import org.apache.hadoop.yarn.util.Records;
 @Stable
 public abstract class ApplicationAttemptId implements
     Comparable<ApplicationAttemptId> {
+  private static Splitter _spliter = Splitter.on('_').trimResults();
 
   @Private
   @Unstable
-  public static final String appAttemptIdStrPrefix = "appattempt_";
+  public static final String appAttemptIdStrPrefix = "appattempt";
 
   @Public
   @Unstable
@@ -131,6 +136,7 @@ public abstract class ApplicationAttemptId implements
   @Override
   public String toString() {
     StringBuilder sb = new StringBuilder(appAttemptIdStrPrefix);
+    sb.append("_");
     sb.append(this.getApplicationId().getClusterTimestamp()).append("_");
     sb.append(ApplicationId.appIdFormat.get().format(
         this.getApplicationId().getId()));
@@ -139,4 +145,32 @@ public abstract class ApplicationAttemptId implements
   }
 
   protected abstract void build();
+  
+  @Public
+  @Stable
+  public static ApplicationAttemptId fromString(String applicationAttemptIdStr) {
+    Iterator<String> it = _spliter.split(applicationAttemptIdStr).iterator();
+    if (!it.next().equals(appAttemptIdStrPrefix)) {
+      throw new IllegalArgumentException("Invalid AppAttemptId prefix: "
+          + applicationAttemptIdStr);
+    }
+    try {
+      return toApplicationAttemptId(it);
+    } catch (NumberFormatException n) {
+      throw new IllegalArgumentException("Invalid AppAttemptId: "
+          + applicationAttemptIdStr, n);
+    } catch (NoSuchElementException e) {
+      throw new IllegalArgumentException("Invalid AppAttemptId: "
+          + applicationAttemptIdStr, e);
+    }
+  }
+  
+  private static ApplicationAttemptId toApplicationAttemptId(
+      Iterator<String> it) throws NumberFormatException {
+    ApplicationId appId = ApplicationId.newInstance(Long.parseLong(it.next()),
+        Integer.parseInt(it.next()));
+    ApplicationAttemptId appAttemptId =
+        ApplicationAttemptId.newInstance(appId, Integer.parseInt(it.next()));
+    return appAttemptId;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c77a1095/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ApplicationId.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ApplicationId.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ApplicationId.java
index 90214cd..03a77ce 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ApplicationId.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ApplicationId.java
@@ -19,6 +19,8 @@
 package org.apache.hadoop.yarn.api.records;
 
 import java.text.NumberFormat;
+import java.util.Iterator;
+import java.util.NoSuchElementException;
 
 import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.classification.InterfaceAudience.Public;
@@ -26,6 +28,8 @@ import org.apache.hadoop.classification.InterfaceStability.Stable;
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.yarn.util.Records;
 
+import com.google.common.base.Splitter;
+
 /**
  * <p><code>ApplicationId</code> represents the <em>globally unique</em> 
  * identifier for an application.</p>
@@ -38,10 +42,11 @@ import org.apache.hadoop.yarn.util.Records;
 @Public
 @Stable
 public abstract class ApplicationId implements Comparable<ApplicationId> {
+  private static Splitter _spliter = Splitter.on('_').trimResults();
 
   @Private
   @Unstable
-  public static final String appIdStrPrefix = "application_";
+  public static final String appIdStrPrefix = "application";
 
   @Public
   @Unstable
@@ -105,8 +110,35 @@ public abstract class ApplicationId implements Comparable<ApplicationId> {
 
   @Override
   public String toString() {
-    return appIdStrPrefix + this.getClusterTimestamp() + "_"
-        + appIdFormat.get().format(getId());
+    return appIdStrPrefix + "_" + this.getClusterTimestamp() + "_" + appIdFormat
+        .get().format(getId());
+  }
+  
+  private static ApplicationId toApplicationId(
+      Iterator<String> it) throws NumberFormatException {
+    ApplicationId appId = ApplicationId.newInstance(Long.parseLong(it.next()),
+        Integer.parseInt(it.next()));
+    return appId;
+  }
+  
+  @Public
+  @Stable
+  public static ApplicationId fromString(String appIdStr) {
+    Iterator<String> it = _spliter.split((appIdStr)).iterator();
+    if (!it.next().equals(appIdStrPrefix)) {
+      throw new IllegalArgumentException("Invalid ApplicationId prefix: "
+          + appIdStr + ". The valid ApplicationId should start with prefix "
+          + appIdStrPrefix);
+    }
+    try {
+      return toApplicationId(it);
+    } catch (NumberFormatException n) {
+      throw new IllegalArgumentException("Invalid ApplicationId: "
+          + appIdStr, n);
+    } catch (NoSuchElementException e) {
+      throw new IllegalArgumentException("Invalid ApplicationId: "
+          + appIdStr, e);
+    }
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c77a1095/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ContainerId.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ContainerId.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ContainerId.java
index f332651..feddeca 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ContainerId.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ContainerId.java
@@ -42,7 +42,7 @@ public abstract class ContainerId implements Comparable<ContainerId>{
   private static final String CONTAINER_PREFIX = "container";
   private static final String EPOCH_PREFIX = "e";
 
-  @Private
+  @Public
   @Unstable
   public static ContainerId newContainerId(ApplicationAttemptId appAttemptId,
       long containerId) {
@@ -97,7 +97,7 @@ public abstract class ContainerId implements Comparable<ContainerId>{
    */
   @Public
   @Deprecated
-  @Stable
+  @Unstable
   public abstract int getId();
 
   /**
@@ -205,7 +205,7 @@ public abstract class ContainerId implements Comparable<ContainerId>{
   }
 
   @Public
-  @Unstable
+  @Stable
   public static ContainerId fromString(String containerIdStr) {
     Iterator<String> it = _SPLITTER.split(containerIdStr).iterator();
     if (!it.next().equals(CONTAINER_PREFIX)) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c77a1095/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/NodeId.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/NodeId.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/NodeId.java
index c3f8595..a0b87a7 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/NodeId.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/NodeId.java
@@ -20,8 +20,8 @@ package org.apache.hadoop.yarn.api.records;
 
 import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.classification.InterfaceAudience.Public;
-import org.apache.hadoop.classification.InterfaceStability.Stable;
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.classification.InterfaceStability.Stable;
 import org.apache.hadoop.yarn.util.Records;
 
 /**
@@ -35,8 +35,8 @@ import org.apache.hadoop.yarn.util.Records;
 @Stable
 public abstract class NodeId implements Comparable<NodeId> {
 
-  @Private
-  @Unstable
+  @Public
+  @Stable
   public static NodeId newInstance(String host, int port) {
     NodeId nodeId = Records.newRecord(NodeId.class);
     nodeId.setHost(host);
@@ -112,6 +112,23 @@ public abstract class NodeId implements Comparable<NodeId> {
     }
     return hostCompare;
   }
+  
+  @Public
+  @Stable
+  public static NodeId fromString(String nodeIdStr) {
+    String[] parts = nodeIdStr.split(":");
+    if (parts.length != 2) {
+      throw new IllegalArgumentException("Invalid NodeId [" + nodeIdStr
+          + "]. Expected host:port");
+    }
+    try {
+      NodeId nodeId =
+          NodeId.newInstance(parts[0].trim(), Integer.parseInt(parts[1]));
+      return nodeId;
+    } catch (NumberFormatException e) {
+      throw new IllegalArgumentException("Invalid port: " + parts[1], e);
+    }
+  }
 
   protected abstract void build();
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c77a1095/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/URL.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/URL.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/URL.java
index 4261117..aa28585 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/URL.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/URL.java
@@ -18,8 +18,13 @@
 
 package org.apache.hadoop.yarn.api.records;
 
+import java.net.URI;
+import java.net.URISyntaxException;
+
 import org.apache.hadoop.classification.InterfaceAudience.Public;
 import org.apache.hadoop.classification.InterfaceStability.Stable;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
 import org.apache.hadoop.yarn.util.Records;
 
 /**
@@ -119,4 +124,48 @@ public abstract class URL {
   @Public
   @Stable
   public abstract void setFile(String file);
+  
+  @Public
+  @Stable
+  public Path toPath() throws URISyntaxException {
+    String scheme = getScheme() == null ? "" : getScheme();
+    
+    String authority = "";
+    if (getHost() != null) {
+      authority = getHost();
+      if (getUserInfo() != null) {
+        authority = getUserInfo() + "@" + authority;
+      }
+      if (getPort() > 0) {
+        authority += ":" + getPort();
+      }
+    }
+    
+    return new Path(
+        (new URI(scheme, authority, getFile(), null, null)).normalize());
+  }
+  
+  @Public
+  @Stable
+  public static URL fromURI(URI uri) {
+    URL url =
+        RecordFactoryProvider.getRecordFactory(null).newRecordInstance(
+            URL.class);
+    if (uri.getHost() != null) {
+      url.setHost(uri.getHost());
+    }
+    if (uri.getUserInfo() != null) {
+      url.setUserInfo(uri.getUserInfo());
+    }
+    url.setPort(uri.getPort());
+    url.setScheme(uri.getScheme());
+    url.setFile(uri.getPath());
+    return url;
+  }
+  
+  @Public
+  @Stable
+  public static URL fromPath(Path path) {
+    return fromURI(path.toUri());
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c77a1095/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/main/java/org/apache/hadoop/yarn/applications/distributedshell/ApplicationMaster.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/main/java/org/apache/hadoop/yarn/applications/distributedshell/ApplicationMaster.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/main/java/org/apache/hadoop/yarn/applications/distributedshell/ApplicationMaster.java
index 5e2c90b..703595c 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/main/java/org/apache/hadoop/yarn/applications/distributedshell/ApplicationMaster.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/main/java/org/apache/hadoop/yarn/applications/distributedshell/ApplicationMaster.java
@@ -435,13 +435,13 @@ public class ApplicationMaster {
     if (!envs.containsKey(Environment.CONTAINER_ID.name())) {
       if (cliParser.hasOption("app_attempt_id")) {
         String appIdStr = cliParser.getOptionValue("app_attempt_id", "");
-        appAttemptID = ConverterUtils.toApplicationAttemptId(appIdStr);
+        appAttemptID = ApplicationAttemptId.fromString(appIdStr);
       } else {
         throw new IllegalArgumentException(
             "Application Attempt Id not set in the environment");
       }
     } else {
-      ContainerId containerId = ConverterUtils.toContainerId(envs
+      ContainerId containerId = ContainerId.fromString(envs
           .get(Environment.CONTAINER_ID.name()));
       appAttemptID = containerId.getApplicationAttemptId();
     }
@@ -1048,8 +1048,7 @@ public class ApplicationMaster {
 
         URL yarnUrl = null;
         try {
-          yarnUrl = ConverterUtils.getYarnUrlFromURI(
-            new URI(renamedScriptPath.toString()));
+          yarnUrl = URL.fromURI(new URI(renamedScriptPath.toString()));
         } catch (URISyntaxException e) {
           LOG.error("Error when trying to use shell script path specified"
               + " in env, path=" + renamedScriptPath, e);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c77a1095/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/main/java/org/apache/hadoop/yarn/applications/distributedshell/Client.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/main/java/org/apache/hadoop/yarn/applications/distributedshell/Client.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/main/java/org/apache/hadoop/yarn/applications/distributedshell/Client.java
index 5adc37d..9879b1e 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/main/java/org/apache/hadoop/yarn/applications/distributedshell/Client.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/main/java/org/apache/hadoop/yarn/applications/distributedshell/Client.java
@@ -68,6 +68,7 @@ import org.apache.hadoop.yarn.api.records.QueueACL;
 import org.apache.hadoop.yarn.api.records.QueueInfo;
 import org.apache.hadoop.yarn.api.records.QueueUserACLInfo;
 import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.api.records.URL;
 import org.apache.hadoop.yarn.api.records.YarnApplicationState;
 import org.apache.hadoop.yarn.api.records.YarnClusterMetrics;
 import org.apache.hadoop.yarn.api.records.timeline.TimelineDomain;
@@ -857,7 +858,7 @@ public class Client {
     FileStatus scFileStatus = fs.getFileStatus(dst);
     LocalResource scRsrc =
         LocalResource.newInstance(
-            ConverterUtils.getYarnUrlFromURI(dst.toUri()),
+            URL.fromURI(dst.toUri()),
             LocalResourceType.FILE, LocalResourceVisibility.APPLICATION,
             scFileStatus.getLen(), scFileStatus.getModificationTime());
     localResources.put(fileDstPath, scRsrc);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c77a1095/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/main/java/org/apache/hadoop/yarn/applications/distributedshell/DistributedShellTimelinePlugin.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/main/java/org/apache/hadoop/yarn/applications/distributedshell/DistributedShellTimelinePlugin.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/main/java/org/apache/hadoop/yarn/applications/distributedshell/DistributedShellTimelinePlugin.java
index 55fbd60..119fa6f 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/main/java/org/apache/hadoop/yarn/applications/distributedshell/DistributedShellTimelinePlugin.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/main/java/org/apache/hadoop/yarn/applications/distributedshell/DistributedShellTimelinePlugin.java
@@ -53,7 +53,7 @@ public class DistributedShellTimelinePlugin extends TimelineEntityGroupPlugin {
   public Set<TimelineEntityGroupId> getTimelineEntityGroupId(String entityId,
       String entityType) {
     if (ApplicationMaster.DSEntity.DS_CONTAINER.toString().equals(entityId)) {
-      ContainerId containerId = ConverterUtils.toContainerId(entityId);
+      ContainerId containerId = ContainerId.fromString(entityId);
       ApplicationId appId = containerId.getApplicationAttemptId()
           .getApplicationId();
       return toEntityGroupId(appId.toString());
@@ -69,7 +69,7 @@ public class DistributedShellTimelinePlugin extends TimelineEntityGroupPlugin {
   }
 
   private Set<TimelineEntityGroupId> toEntityGroupId(String strAppId) {
-    ApplicationId appId = ConverterUtils.toApplicationId(strAppId);
+    ApplicationId appId = ApplicationId.fromString(strAppId);
     TimelineEntityGroupId groupId = TimelineEntityGroupId.newInstance(
         appId, ApplicationMaster.CONTAINER_ENTITY_GROUP_ID);
     Set<TimelineEntityGroupId> result = new HashSet<>();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c77a1095/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/test/java/org/apache/hadoop/yarn/applications/distributedshell/TestDistributedShell.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/test/java/org/apache/hadoop/yarn/applications/distributedshell/TestDistributedShell.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/test/java/org/apache/hadoop/yarn/applications/distributedshell/TestDistributedShell.java
index 2b46fca..9448cf1 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/test/java/org/apache/hadoop/yarn/applications/distributedshell/TestDistributedShell.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/test/java/org/apache/hadoop/yarn/applications/distributedshell/TestDistributedShell.java
@@ -371,8 +371,8 @@ public class TestDistributedShell {
     }
     String currAttemptEntityId
         = entitiesAttempts.getEntities().get(0).getEntityId();
-    ApplicationAttemptId attemptId
-        = ConverterUtils.toApplicationAttemptId(currAttemptEntityId);
+    ApplicationAttemptId attemptId = ApplicationAttemptId.fromString(
+        currAttemptEntityId);
     NameValuePair primaryFilter = new NameValuePair(
         ApplicationMaster.APPID_TIMELINE_FILTER_NAME,
         attemptId.getApplicationId().toString());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c77a1095/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/ApplicationCLI.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/ApplicationCLI.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/ApplicationCLI.java
index d9e9fa6..865ce00 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/ApplicationCLI.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/ApplicationCLI.java
@@ -301,7 +301,7 @@ public class ApplicationCLI extends YarnCLI {
    */
   private void signalToContainer(String containerIdStr,
       SignalContainerCommand command) throws YarnException, IOException {
-    ContainerId containerId = ConverterUtils.toContainerId(containerIdStr);
+    ContainerId containerId = ContainerId.fromString(containerIdStr);
     sysout.println("Signalling container " + containerIdStr);
     client.signalToContainer(containerId, command);
   }
@@ -327,8 +327,8 @@ public class ApplicationCLI extends YarnCLI {
       throws YarnException, IOException {
     ApplicationAttemptReport appAttemptReport = null;
     try {
-      appAttemptReport = client.getApplicationAttemptReport(ConverterUtils
-          .toApplicationAttemptId(applicationAttemptId));
+      appAttemptReport = client.getApplicationAttemptReport(
+          ApplicationAttemptId.fromString(applicationAttemptId));
     } catch (ApplicationNotFoundException e) {
       sysout.println("Application for AppAttempt with id '"
           + applicationAttemptId + "' doesn't exist in RM or Timeline Server.");
@@ -384,8 +384,7 @@ public class ApplicationCLI extends YarnCLI {
       IOException {
     ContainerReport containerReport = null;
     try {
-      containerReport = client.getContainerReport((ConverterUtils
-          .toContainerId(containerId)));
+      containerReport = client.getContainerReport(ContainerId.fromString(containerId));
     } catch (ApplicationNotFoundException e) {
       sysout.println("Application for Container with id '" + containerId
           + "' doesn't exist in RM or Timeline Server.");
@@ -515,7 +514,7 @@ public class ApplicationCLI extends YarnCLI {
    */
   private void killApplication(String applicationId) throws YarnException,
       IOException {
-    ApplicationId appId = ConverterUtils.toApplicationId(applicationId);
+    ApplicationId appId = ApplicationId.fromString(applicationId);
     ApplicationReport  appReport = null;
     try {
       appReport = client.getApplicationReport(appId);
@@ -540,7 +539,7 @@ public class ApplicationCLI extends YarnCLI {
    */
   private void moveApplicationAcrossQueues(String applicationId, String queue)
       throws YarnException, IOException {
-    ApplicationId appId = ConverterUtils.toApplicationId(applicationId);
+    ApplicationId appId = ApplicationId.fromString(applicationId);
     ApplicationReport appReport = client.getApplicationReport(appId);
     if (appReport.getYarnApplicationState() == YarnApplicationState.FINISHED
         || appReport.getYarnApplicationState() == YarnApplicationState.KILLED
@@ -565,7 +564,7 @@ public class ApplicationCLI extends YarnCLI {
       IOException {
     ApplicationId appId;
     ApplicationAttemptId attId;
-    attId = ConverterUtils.toApplicationAttemptId(attemptId);
+    attId = ApplicationAttemptId.fromString(attemptId);
     appId = attId.getApplicationId();
 
     sysout.println("Failing attempt " + attId + " of application " + appId);
@@ -583,8 +582,8 @@ public class ApplicationCLI extends YarnCLI {
       throws YarnException, IOException {
     ApplicationReport appReport = null;
     try {
-      appReport = client.getApplicationReport(ConverterUtils
-          .toApplicationId(applicationId));
+      appReport = client.getApplicationReport(
+          ApplicationId.fromString(applicationId));
     } catch (ApplicationNotFoundException e) {
       sysout.println("Application with id '" + applicationId
           + "' doesn't exist in RM or Timeline Server.");
@@ -684,7 +683,7 @@ public class ApplicationCLI extends YarnCLI {
         new OutputStreamWriter(sysout, Charset.forName("UTF-8")));
 
     List<ApplicationAttemptReport> appAttemptsReport = client
-        .getApplicationAttempts(ConverterUtils.toApplicationId(applicationId));
+        .getApplicationAttempts(ApplicationId.fromString(applicationId));
     writer.println("Total number of application attempts " + ":"
         + appAttemptsReport.size());
     writer.printf(APPLICATION_ATTEMPTS_PATTERN, "ApplicationAttempt-Id",
@@ -711,8 +710,8 @@ public class ApplicationCLI extends YarnCLI {
     PrintWriter writer = new PrintWriter(
         new OutputStreamWriter(sysout, Charset.forName("UTF-8")));
 
-    List<ContainerReport> appsReport = client
-        .getContainers(ConverterUtils.toApplicationAttemptId(appAttemptId));
+    List<ContainerReport> appsReport = client.getContainers(
+        ApplicationAttemptId.fromString(appAttemptId));
     writer.println("Total number of containers " + ":" + appsReport.size());
     writer.printf(CONTAINER_PATTERN, "Container-Id", "Start Time",
         "Finish Time", "State", "Host", "Node Http Address", "LOG-URL");
@@ -735,7 +734,7 @@ public class ApplicationCLI extends YarnCLI {
    */
   private void updateApplicationPriority(String applicationId, String priority)
       throws YarnException, IOException {
-    ApplicationId appId = ConverterUtils.toApplicationId(applicationId);
+    ApplicationId appId = ApplicationId.fromString(applicationId);
     Priority newAppPriority = Priority.newInstance(Integer.parseInt(priority));
     sysout.println("Updating priority of an application " + applicationId);
     Priority updateApplicationPriority =

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c77a1095/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/LogsCLI.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/LogsCLI.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/LogsCLI.java
index d62ee5e..4fdb57b 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/LogsCLI.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/LogsCLI.java
@@ -153,7 +153,7 @@ public class LogsCLI extends Configured implements Tool {
 
     ApplicationId appId = null;
     try {
-      appId = ConverterUtils.toApplicationId(appIdStr);
+      appId = ApplicationId.fromString(appIdStr);
     } catch (Exception e) {
       System.err.println("Invalid ApplicationId specified");
       return -1;
@@ -456,8 +456,8 @@ public class LogsCLI extends Configured implements Tool {
       throws YarnException, IOException {
     YarnClient yarnClient = createYarnClient();
     try {
-      return yarnClient.getContainerReport(ConverterUtils
-        .toContainerId(containerIdStr));
+      return yarnClient.getContainerReport(
+          ContainerId.fromString(containerIdStr));
     } finally {
       yarnClient.close();
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c77a1095/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/NodeCLI.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/NodeCLI.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/NodeCLI.java
index a89551f..f51fee9 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/NodeCLI.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/NodeCLI.java
@@ -243,7 +243,7 @@ public class NodeCLI extends YarnCLI {
    */
   private void printNodeStatus(String nodeIdStr) throws YarnException,
       IOException {
-    NodeId nodeId = ConverterUtils.toNodeId(nodeIdStr);
+    NodeId nodeId = NodeId.fromString(nodeIdStr);
     List<NodeReport> nodesReport = client.getNodeReports();
     // Use PrintWriter.println, which uses correct platform line ending.
     ByteArrayOutputStream baos = new ByteArrayOutputStream();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c77a1095/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/RMAdminCLI.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/RMAdminCLI.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/RMAdminCLI.java
index d407c20..aa7fc30 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/RMAdminCLI.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/RMAdminCLI.java
@@ -427,7 +427,7 @@ public class RMAdminCLI extends HAAdmin {
     ResourceManagerAdministrationProtocol adminProtocol = createAdminProtocol();
     UpdateNodeResourceRequest request =
       recordFactory.newRecordInstance(UpdateNodeResourceRequest.class);
-    NodeId nodeId = ConverterUtils.toNodeId(nodeIdStr);
+    NodeId nodeId = NodeId.fromString(nodeIdStr);
     
     Resource resource = Resources.createResource(memSize, cores);
     Map<NodeId, ResourceOption> resourceMap =

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c77a1095/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestRMAdminCLI.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestRMAdminCLI.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestRMAdminCLI.java
index 057594d..1551333 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestRMAdminCLI.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestRMAdminCLI.java
@@ -222,7 +222,7 @@ public class TestRMAdminCLI {
     verify(admin).updateNodeResource(argument.capture());
     UpdateNodeResourceRequest request = argument.getValue();
     Map<NodeId, ResourceOption> resourceMap = request.getNodeResourceMap();
-    NodeId nodeId = ConverterUtils.toNodeId(nodeIdStr);
+    NodeId nodeId = NodeId.fromString(nodeIdStr);
     Resource expectedResource = Resources.createResource(memSize, cores);
     ResourceOption resource = resourceMap.get(nodeId);
     assertNotNull("resource for " + nodeIdStr + " shouldn't be null.",

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c77a1095/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/AggregatedLogDeletionService.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/AggregatedLogDeletionService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/AggregatedLogDeletionService.java
index 4c1d152..a80f9d7 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/AggregatedLogDeletionService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/AggregatedLogDeletionService.java
@@ -99,7 +99,7 @@ public class AggregatedLogDeletionService extends AbstractService {
           if(appDir.isDirectory() && 
               appDir.getModificationTime() < cutoffMillis) {
             boolean appTerminated =
-                isApplicationTerminated(ConverterUtils.toApplicationId(appDir
+                isApplicationTerminated(ApplicationId.fromString(appDir
                   .getPath().getName()), rmClient);
             if(appTerminated && shouldDeleteLogDir(appDir, cutoffMillis, fs)) {
               try {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c77a1095/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/AggregatedLogFormat.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/AggregatedLogFormat.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/AggregatedLogFormat.java
index 98ffce1..8b213d5 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/AggregatedLogFormat.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/AggregatedLogFormat.java
@@ -209,14 +209,11 @@ public class AggregatedLogFormat {
     public Set<File> getPendingLogFilesToUploadForThisContainer() {
       Set<File> pendingUploadFiles = new HashSet<File>();
       for (String rootLogDir : this.rootLogDirs) {
-        File appLogDir =
-            new File(rootLogDir, 
-                ConverterUtils.toString(
-                    this.containerId.getApplicationAttemptId().
-                        getApplicationId())
-                );
+        File appLogDir = new File(rootLogDir,
+            this.containerId.getApplicationAttemptId().
+                getApplicationId().toString());
         File containerLogDir =
-            new File(appLogDir, ConverterUtils.toString(this.containerId));
+            new File(appLogDir, this.containerId.toString());
 
         if (!containerLogDir.isDirectory()) {
           continue; // ContainerDir may have been deleted by the user.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c77a1095/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/LogCLIHelpers.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/LogCLIHelpers.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/LogCLIHelpers.java
index 3811054..26b2b01 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/LogCLIHelpers.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/LogCLIHelpers.java
@@ -59,7 +59,7 @@ public class LogCLIHelpers implements Configurable {
   public int dumpAContainersLogs(String appId, String containerId,
       String nodeId, String jobOwner) throws IOException {
     ContainerLogsRequest options = new ContainerLogsRequest();
-    options.setAppId(ConverterUtils.toApplicationId(appId));
+    options.setAppId(ApplicationId.fromString(appId));
     options.setContainerId(containerId);
     options.setNodeId(nodeId);
     options.setAppOwner(jobOwner);


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


[07/39] hadoop git commit: YARN-1942. Deprecate toString/fromString methods from ConverterUtils and move them to records classes like ContainerId/ApplicationId, etc. (wangda)

Posted by ae...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/c77a1095/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/ConverterUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/ConverterUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/ConverterUtils.java
index acd29fb..67bc2b7 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/ConverterUtils.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/ConverterUtils.java
@@ -18,18 +18,13 @@
 
 package org.apache.hadoop.yarn.util;
 
-import static org.apache.hadoop.yarn.util.StringHelper._split;
-
 import java.net.InetSocketAddress;
 import java.net.URI;
 import java.net.URISyntaxException;
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.NoSuchElementException;
 
 import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceAudience.Public;
+import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.security.SecurityUtil;
@@ -41,7 +36,6 @@ import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.URL;
 import org.apache.hadoop.yarn.factories.RecordFactory;
-import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
 
 
 /**
@@ -49,7 +43,7 @@ import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
  * from/to 'serializableFormat' to/from hadoop/nativejava data structures.
  *
  */
-@Private
+@Public
 public class ConverterUtils {
 
   public static final String APPLICATION_PREFIX = "application";
@@ -58,174 +52,114 @@ public class ConverterUtils {
 
   /**
    * return a hadoop path from a given url
+   * This method is deprecated, use {@link URL#toPath()} instead.
    * 
    * @param url
    *          url to convert
    * @return path from {@link URL}
    * @throws URISyntaxException
    */
+  @Public
+  @Deprecated
   public static Path getPathFromYarnURL(URL url) throws URISyntaxException {
-    String scheme = url.getScheme() == null ? "" : url.getScheme();
-    
-    String authority = "";
-    if (url.getHost() != null) {
-      authority = url.getHost();
-      if (url.getUserInfo() != null) {
-        authority = url.getUserInfo() + "@" + authority;
-      }
-      if (url.getPort() > 0) {
-        authority += ":" + url.getPort();
-      }
-    }
-    
-    return new Path(
-        (new URI(scheme, authority, url.getFile(), null, null)).normalize());
+    return url.toPath();
   }
-  
-  /**
-   * change from CharSequence to string for map key and value
-   * @param env map for converting
-   * @return string,string map
-   */
-  public static Map<String, String> convertToString(
-      Map<CharSequence, CharSequence> env) {
-    
-    Map<String, String> stringMap = new HashMap<String, String>();
-    for (Entry<CharSequence, CharSequence> entry: env.entrySet()) {
-      stringMap.put(entry.getKey().toString(), entry.getValue().toString());
-    }
-    return stringMap;
-   }
 
+  /*
+   * This method is deprecated, use {@link URL#fromPath(Path)} instead.
+   */
+  @Public
+  @Deprecated
   public static URL getYarnUrlFromPath(Path path) {
-    return getYarnUrlFromURI(path.toUri());
+    return URL.fromPath(path);
   }
   
+  /*
+   * This method is deprecated, use {@link URL#fromURI(URI)} instead.
+   */
+  @Public
+  @Deprecated
   public static URL getYarnUrlFromURI(URI uri) {
-    URL url = RecordFactoryProvider.getRecordFactory(null).newRecordInstance(URL.class);
-    if (uri.getHost() != null) {
-      url.setHost(uri.getHost());
-    }
-    if (uri.getUserInfo() != null) {
-      url.setUserInfo(uri.getUserInfo());
-    }
-    url.setPort(uri.getPort());
-    url.setScheme(uri.getScheme());
-    url.setFile(uri.getPath());
-    return url;
+    return URL.fromURI(uri);
   }
 
+  /*
+   * This method is deprecated, use {@link ApplicationId#toString()} instead.
+   */
+  @Public
+  @Deprecated
   public static String toString(ApplicationId appId) {
     return appId.toString();
   }
 
+  /*
+   * This method is deprecated, use {@link ApplicationId#fromString(String)}
+   * instead.
+   */
+  @Public
+  @Deprecated
   public static ApplicationId toApplicationId(RecordFactory recordFactory,
-      String appIdStr) {
-    Iterator<String> it = _split(appIdStr).iterator();
-    if (!it.next().equals(APPLICATION_PREFIX)) {
-      throw new IllegalArgumentException("Invalid ApplicationId prefix: "
-          + appIdStr + ". The valid ApplicationId should start with prefix "
-          + APPLICATION_PREFIX);
-    }
-    try {
-      return toApplicationId(recordFactory, it);
-    } catch (NumberFormatException n) {
-      throw new IllegalArgumentException("Invalid ApplicationId: " + appIdStr,
-          n);
-    } catch (NoSuchElementException e) {
-      throw new IllegalArgumentException("Invalid ApplicationId: " + appIdStr,
-          e);
-    }
-  }
-
-  private static ApplicationId toApplicationId(RecordFactory recordFactory,
-      Iterator<String> it) {
-    ApplicationId appId = ApplicationId.newInstance(Long.parseLong(it.next()),
-        Integer.parseInt(it.next()));
-    return appId;
-  }
-
-  private static ApplicationAttemptId toApplicationAttemptId(
-      Iterator<String> it) throws NumberFormatException {
-    ApplicationId appId = ApplicationId.newInstance(Long.parseLong(it.next()),
-        Integer.parseInt(it.next()));
-    ApplicationAttemptId appAttemptId =
-        ApplicationAttemptId.newInstance(appId, Integer.parseInt(it.next()));
-    return appAttemptId;
-  }
-
-  private static ApplicationId toApplicationId(
-      Iterator<String> it) throws NumberFormatException {
-    ApplicationId appId = ApplicationId.newInstance(Long.parseLong(it.next()),
-        Integer.parseInt(it.next()));
-    return appId;
+      String applicationIdStr) {
+    return ApplicationId.fromString(applicationIdStr);
   }
 
+  /*
+   * This method is deprecated, use {@link ContainerId#toString()} instead.
+   */
+  @Public
+  @Deprecated
   public static String toString(ContainerId cId) {
     return cId == null ? null : cId.toString();
   }
-  
+
+  @Private
+  @InterfaceStability.Unstable
   public static NodeId toNodeIdWithDefaultPort(String nodeIdStr) {
     if (nodeIdStr.indexOf(":") < 0) {
-      return toNodeId(nodeIdStr + ":0");
+      return NodeId.fromString(nodeIdStr + ":0");
     }
-    return toNodeId(nodeIdStr);
+    return NodeId.fromString(nodeIdStr);
   }
 
+  /*
+   * This method is deprecated, use {@link NodeId#fromString(String)} instead.
+   */
+  @Public
+  @Deprecated
   public static NodeId toNodeId(String nodeIdStr) {
-    String[] parts = nodeIdStr.split(":");
-    if (parts.length != 2) {
-      throw new IllegalArgumentException("Invalid NodeId [" + nodeIdStr
-          + "]. Expected host:port");
-    }
-    try {
-      NodeId nodeId =
-          NodeId.newInstance(parts[0].trim(), Integer.parseInt(parts[1]));
-      return nodeId;
-    } catch (NumberFormatException e) {
-      throw new IllegalArgumentException("Invalid port: " + parts[1], e);
-    }
+    return NodeId.fromString(nodeIdStr);
   }
 
+  /*
+   * This method is deprecated, use {@link ContainerId#fromString(String)}
+   * instead.
+   */
+  @Public
+  @Deprecated
   public static ContainerId toContainerId(String containerIdStr) {
     return ContainerId.fromString(containerIdStr);
   }
-
+  
+  /*
+   * This method is deprecated, use {@link ApplicationAttemptId#toString()}
+   * instead.
+   */
+  @Public
+  @Deprecated
   public static ApplicationAttemptId toApplicationAttemptId(
-      String applicationAttmeptIdStr) {
-    Iterator<String> it = _split(applicationAttmeptIdStr).iterator();
-    if (!it.next().equals(APPLICATION_ATTEMPT_PREFIX)) {
-      throw new IllegalArgumentException("Invalid AppAttemptId prefix: "
-          + applicationAttmeptIdStr);
-    }
-    try {
-      return toApplicationAttemptId(it);
-    } catch (NumberFormatException n) {
-      throw new IllegalArgumentException("Invalid AppAttemptId: "
-          + applicationAttmeptIdStr, n);
-    } catch (NoSuchElementException e) {
-      throw new IllegalArgumentException("Invalid AppAttemptId: "
-          + applicationAttmeptIdStr, e);
-    }
+      String applicationAttemptIdStr) {
+    return ApplicationAttemptId.fromString(applicationAttemptIdStr);
   }
   
+  /*
+   * This method is deprecated, use {@link ApplicationId#fromString(String)}
+   * instead.
+   */
+  @Public
+  @Deprecated
   public static ApplicationId toApplicationId(
       String appIdStr) {
-    Iterator<String> it = _split(appIdStr).iterator();
-    if (!it.next().equals(APPLICATION_PREFIX)) {
-      throw new IllegalArgumentException("Invalid ApplicationId prefix: "
-          + appIdStr + ". The valid ApplicationId should start with prefix "
-          + APPLICATION_PREFIX);
-    }
-    try {
-      return toApplicationId(it);
-    } catch (NumberFormatException n) {
-      throw new IllegalArgumentException("Invalid ApplicationId: "
-          + appIdStr, n);
-    } catch (NoSuchElementException e) {
-      throw new IllegalArgumentException("Invalid ApplicationId: "
-          + appIdStr, e);
-    }
+    return ApplicationId.fromString(appIdStr);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c77a1095/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/FSDownload.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/FSDownload.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/FSDownload.java
index bd9c907..de18dc6 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/FSDownload.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/FSDownload.java
@@ -346,7 +346,7 @@ public class FSDownload implements Callable<Path> {
   public Path call() throws Exception {
     final Path sCopy;
     try {
-      sCopy = ConverterUtils.getPathFromYarnURL(resource.getResource());
+      sCopy = resource.getResource().toPath();
     } catch (URISyntaxException e) {
       throw new IOException("Invalid resource", e);
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c77a1095/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/log/AggregatedLogsBlock.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/log/AggregatedLogsBlock.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/log/AggregatedLogsBlock.java
index 2fc8dfc..1da6e23 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/log/AggregatedLogsBlock.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/log/AggregatedLogsBlock.java
@@ -290,7 +290,7 @@ public class AggregatedLogsBlock extends HtmlBlock {
     }
     ContainerId containerId = null;
     try {
-      containerId = ConverterUtils.toContainerId(containerIdStr);
+      containerId = ContainerId.fromString(containerIdStr);
     } catch (IllegalArgumentException e) {
       html.h1()
           ._("Cannot get container logs for invalid containerId: "
@@ -308,7 +308,7 @@ public class AggregatedLogsBlock extends HtmlBlock {
     }
     NodeId nodeId = null;
     try {
-      nodeId = ConverterUtils.toNodeId(nodeIdStr);
+      nodeId = NodeId.fromString(nodeIdStr);
     } catch (IllegalArgumentException e) {
       html.h1()._("Cannot get container logs. Invalid nodeId: " + nodeIdStr)
           ._();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c77a1095/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/util/WebAppUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/util/WebAppUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/util/WebAppUtils.java
index 3aa773a..6245541 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/util/WebAppUtils.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/util/WebAppUtils.java
@@ -392,7 +392,7 @@ public class WebAppUtils {
     }
     ApplicationId aid = null;
     try {
-      aid = ConverterUtils.toApplicationId(recordFactory, appId);
+      aid = ApplicationId.fromString(appId);
     } catch (Exception e) {
       throw new BadRequestException(e);
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c77a1095/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/util/TestConverterUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/util/TestConverterUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/util/TestConverterUtils.java
index 3cec38b..077558b 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/util/TestConverterUtils.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/util/TestConverterUtils.java
@@ -34,55 +34,56 @@ public class TestConverterUtils {
   @Test
   public void testConvertUrlWithNoPort() throws URISyntaxException {
     Path expectedPath = new Path("hdfs://foo.com");
-    URL url = ConverterUtils.getYarnUrlFromPath(expectedPath);
-    Path actualPath = ConverterUtils.getPathFromYarnURL(url);
+    URL url = URL.fromPath(expectedPath);
+    Path actualPath = url.toPath();
     assertEquals(expectedPath, actualPath);
   }
 
   @Test
   public void testConvertUrlWithUserinfo() throws URISyntaxException {
     Path expectedPath = new Path("foo://username:password@example.com:8042");
-    URL url = ConverterUtils.getYarnUrlFromPath(expectedPath);
-    Path actualPath = ConverterUtils.getPathFromYarnURL(url);
+    URL url = URL.fromPath(expectedPath);
+    Path actualPath = url.toPath();
     assertEquals(expectedPath, actualPath);
   }
   
   @Test
   public void testContainerId() throws URISyntaxException {
     ContainerId id = TestContainerId.newContainerId(0, 0, 0, 0);
-    String cid = ConverterUtils.toString(id);
+    String cid = id.toString();
     assertEquals("container_0_0000_00_000000", cid);
-    ContainerId gen = ConverterUtils.toContainerId(cid);
+    ContainerId gen = ContainerId.fromString(cid);
     assertEquals(gen, id);
   }
 
   @Test
   public void testContainerIdWithEpoch() throws URISyntaxException {
     ContainerId id = TestContainerId.newContainerId(0, 0, 0, 25645811);
-    String cid = ConverterUtils.toString(id);
+    String cid = id.toString();
     assertEquals("container_0_0000_00_25645811", cid);
-    ContainerId gen = ConverterUtils.toContainerId(cid);
+    ContainerId gen = ContainerId.fromString(cid);
     assertEquals(gen.toString(), id.toString());
 
     long ts = System.currentTimeMillis();
     ContainerId id2 =
         TestContainerId.newContainerId(36473, 4365472, ts, 4298334883325L);
-    String cid2 = ConverterUtils.toString(id2);
+    String cid2 = id2.toString();
     assertEquals(
         "container_e03_" + ts + "_36473_4365472_999799999997", cid2);
-    ContainerId gen2 = ConverterUtils.toContainerId(cid2);
+    ContainerId gen2 = ContainerId.fromString(cid2);
     assertEquals(gen2.toString(), id2.toString());
 
     ContainerId id3 =
         TestContainerId.newContainerId(36473, 4365472, ts, 844424930131965L);
-    String cid3 = ConverterUtils.toString(id3);
+    String cid3 = id3.toString();
     assertEquals(
         "container_e767_" + ts + "_36473_4365472_1099511627773", cid3);
-    ContainerId gen3 = ConverterUtils.toContainerId(cid3);
+    ContainerId gen3 = ContainerId.fromString(cid3);
     assertEquals(gen3.toString(), id3.toString());
   }
 
   @Test
+  @SuppressWarnings("deprecation")
   public void testContainerIdNull() throws URISyntaxException {
     assertNull(ConverterUtils.toString((ContainerId)null));
   }  
@@ -101,16 +102,19 @@ public class TestConverterUtils {
   }
 
   @Test(expected = IllegalArgumentException.class)
+  @SuppressWarnings("deprecation")
   public void testInvalidContainerId() {
-    ConverterUtils.toContainerId("container_e20_1423221031460_0003_01");
+    ContainerId.fromString("container_e20_1423221031460_0003_01");
   }
 
   @Test(expected = IllegalArgumentException.class)
+  @SuppressWarnings("deprecation")
   public void testInvalidAppattemptId() {
     ConverterUtils.toApplicationAttemptId("appattempt_1423221031460");
   }
 
   @Test(expected = IllegalArgumentException.class)
+  @SuppressWarnings("deprecation")
   public void testApplicationId() {
     ConverterUtils.toApplicationId("application_1423221031460");
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c77a1095/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/util/TestFSDownload.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/util/TestFSDownload.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/util/TestFSDownload.java
index 376b27d..877dd08 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/util/TestFSDownload.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/util/TestFSDownload.java
@@ -53,6 +53,7 @@ import java.util.zip.ZipEntry;
 import java.util.zip.ZipOutputStream;
 
 import org.apache.hadoop.util.concurrent.HadoopExecutors;
+import org.apache.hadoop.yarn.api.records.URL;
 import org.junit.Assert;
 
 import org.apache.commons.compress.archivers.tar.TarArchiveEntry;
@@ -103,7 +104,7 @@ public class TestFSDownload {
       Random r, LocalResourceVisibility vis) throws IOException {
     createFile(files, p, len, r);
     LocalResource ret = recordFactory.newRecordInstance(LocalResource.class);
-    ret.setResource(ConverterUtils.getYarnUrlFromPath(p));
+    ret.setResource(URL.fromPath(p));
     ret.setSize(len);
     ret.setType(LocalResourceType.FILE);
     ret.setVisibility(vis);
@@ -134,7 +135,7 @@ public class TestFSDownload {
     LOG.info("Done writing jar stream ");
     out.close();
     LocalResource ret = recordFactory.newRecordInstance(LocalResource.class);
-    ret.setResource(ConverterUtils.getYarnUrlFromPath(p));
+    ret.setResource(URL.fromPath(p));
     FileStatus status = files.getFileStatus(p);
     ret.setSize(status.getLen());
     ret.setTimestamp(status.getModificationTime());
@@ -162,7 +163,7 @@ public class TestFSDownload {
     out.close();
 
     LocalResource ret = recordFactory.newRecordInstance(LocalResource.class);
-    ret.setResource(ConverterUtils.getYarnUrlFromPath(new Path(p.toString()
+    ret.setResource(URL.fromPath(new Path(p.toString()
         + ".tar")));
     ret.setSize(len);
     ret.setType(LocalResourceType.ARCHIVE);
@@ -190,7 +191,7 @@ public class TestFSDownload {
     out.close();
 
     LocalResource ret = recordFactory.newRecordInstance(LocalResource.class);
-    ret.setResource(ConverterUtils.getYarnUrlFromPath(new Path(p.toString()
+    ret.setResource(URL.fromPath(new Path(p.toString()
         + ".tar.gz")));
     ret.setSize(len);
     ret.setType(LocalResourceType.ARCHIVE);
@@ -216,7 +217,7 @@ public class TestFSDownload {
     out.close();
 
     LocalResource ret = recordFactory.newRecordInstance(LocalResource.class);
-    ret.setResource(ConverterUtils.getYarnUrlFromPath(new Path(p.toString()
+    ret.setResource(URL.fromPath(new Path(p.toString()
         + ".jar")));
     ret.setSize(len);
     ret.setType(LocalResourceType.ARCHIVE);
@@ -242,7 +243,7 @@ public class TestFSDownload {
     out.close();
 
     LocalResource ret = recordFactory.newRecordInstance(LocalResource.class);
-    ret.setResource(ConverterUtils.getYarnUrlFromPath(new Path(p.toString()
+    ret.setResource(URL.fromPath(new Path(p.toString()
         + ".ZIP")));
     ret.setSize(len);
     ret.setType(LocalResourceType.ARCHIVE);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c77a1095/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/ApplicationHistoryManagerOnTimelineStore.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/ApplicationHistoryManagerOnTimelineStore.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/ApplicationHistoryManagerOnTimelineStore.java
index aedf6f6..84d4543 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/ApplicationHistoryManagerOnTimelineStore.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/ApplicationHistoryManagerOnTimelineStore.java
@@ -278,7 +278,7 @@ public class ApplicationHistoryManagerOnTimelineStore extends AbstractService
       }
       if (field == ApplicationReportField.USER_AND_ACLS) {
         return new ApplicationReportExt(ApplicationReport.newInstance(
-            ConverterUtils.toApplicationId(entity.getEntityId()),
+            ApplicationId.fromString(entity.getEntityId()),
             latestApplicationAttemptId, user, queue, name, null, -1, null,
             state, diagnosticsInfo, null, createdTime, finishedTime,
             finalStatus, null, null, progress, type, null, appTags,
@@ -394,13 +394,10 @@ public class ApplicationHistoryManagerOnTimelineStore extends AbstractService
           }
           if (eventInfo
               .containsKey(ApplicationMetricsConstants.LATEST_APP_ATTEMPT_EVENT_INFO)) {
-            latestApplicationAttemptId =
-                ConverterUtils
-                    .toApplicationAttemptId(
-                    eventInfo
-                        .get(
-                            ApplicationMetricsConstants.LATEST_APP_ATTEMPT_EVENT_INFO)
-                        .toString());
+            latestApplicationAttemptId = ApplicationAttemptId.fromString(
+                eventInfo.get(
+                    ApplicationMetricsConstants.LATEST_APP_ATTEMPT_EVENT_INFO)
+                    .toString());
           }
           if (eventInfo
               .containsKey(ApplicationMetricsConstants.DIAGNOSTICS_INFO_EVENT_INFO)) {
@@ -426,7 +423,7 @@ public class ApplicationHistoryManagerOnTimelineStore extends AbstractService
       }
     }
     return new ApplicationReportExt(ApplicationReport.newInstance(
-        ConverterUtils.toApplicationId(entity.getEntityId()),
+        ApplicationId.fromString(entity.getEntityId()),
         latestApplicationAttemptId, user, queue, name, null, -1, null, state,
         diagnosticsInfo, null, createdTime, finishedTime, finalStatus,
         appResources, null, progress, type, null, appTags, unmanagedApplication,
@@ -471,7 +468,7 @@ public class ApplicationHistoryManagerOnTimelineStore extends AbstractService
           if (eventInfo
               .containsKey(AppAttemptMetricsConstants.MASTER_CONTAINER_EVENT_INFO)) {
             amContainerId =
-                ConverterUtils.toContainerId(eventInfo.get(
+                ContainerId.fromString(eventInfo.get(
                     AppAttemptMetricsConstants.MASTER_CONTAINER_EVENT_INFO)
                     .toString());
           }
@@ -513,7 +510,7 @@ public class ApplicationHistoryManagerOnTimelineStore extends AbstractService
           if (eventInfo
               .containsKey(AppAttemptMetricsConstants.MASTER_CONTAINER_EVENT_INFO)) {
             amContainerId =
-                ConverterUtils.toContainerId(eventInfo.get(
+                ContainerId.fromString(eventInfo.get(
                     AppAttemptMetricsConstants.MASTER_CONTAINER_EVENT_INFO)
                     .toString());
           }
@@ -521,7 +518,7 @@ public class ApplicationHistoryManagerOnTimelineStore extends AbstractService
       }
     }
     return ApplicationAttemptReport.newInstance(
-        ConverterUtils.toApplicationAttemptId(entity.getEntityId()),
+        ApplicationAttemptId.fromString(entity.getEntityId()),
         host, rpcPort, trackingUrl, originalTrackingUrl, diagnosticsInfo,
         state, amContainerId);
   }
@@ -610,7 +607,7 @@ public class ApplicationHistoryManagerOnTimelineStore extends AbstractService
       }
     }
     ContainerId containerId =
-        ConverterUtils.toContainerId(entity.getEntityId());
+        ContainerId.fromString(entity.getEntityId());
     String logUrl = null;
     NodeId allocatedNode = null;
     if (allocatedHost != null) {
@@ -623,7 +620,7 @@ public class ApplicationHistoryManagerOnTimelineStore extends AbstractService
           user);
     }
     return ContainerReport.newInstance(
-        ConverterUtils.toContainerId(entity.getEntityId()),
+        ContainerId.fromString(entity.getEntityId()),
         Resource.newInstance(allocatedMem, allocatedVcore), allocatedNode,
         Priority.newInstance(allocatedPriority),
         createdTime, finishedTime, diagnosticsInfo, logUrl, exitStatus, state,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c77a1095/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/FileSystemApplicationHistoryStore.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/FileSystemApplicationHistoryStore.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/FileSystemApplicationHistoryStore.java
index c340b19..295b8ab 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/FileSystemApplicationHistoryStore.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/FileSystemApplicationHistoryStore.java
@@ -204,7 +204,7 @@ public class FileSystemApplicationHistoryStore extends AbstractService
     FileStatus[] files = fs.listStatus(rootDirPath);
     for (FileStatus file : files) {
       ApplicationId appId =
-          ConverterUtils.toApplicationId(file.getPath().getName());
+          ApplicationId.fromString(file.getPath().getName());
       try {
         ApplicationHistoryData historyData = getApplication(appId);
         if (historyData != null) {
@@ -231,8 +231,8 @@ public class FileSystemApplicationHistoryStore extends AbstractService
         HistoryFileReader.Entry entry = hfReader.next();
         if (entry.key.id.startsWith(
             ConverterUtils.APPLICATION_ATTEMPT_PREFIX)) {
-          ApplicationAttemptId appAttemptId = 
-              ConverterUtils.toApplicationAttemptId(entry.key.id);
+          ApplicationAttemptId appAttemptId = ApplicationAttemptId.fromString(
+              entry.key.id);
           if (appAttemptId.getApplicationId().equals(appId)) {
             ApplicationAttemptHistoryData historyData = 
                 historyDataMap.get(appAttemptId);
@@ -385,7 +385,7 @@ public class FileSystemApplicationHistoryStore extends AbstractService
         HistoryFileReader.Entry entry = hfReader.next();
         if (entry.key.id.startsWith(ConverterUtils.CONTAINER_PREFIX)) {
           ContainerId containerId =
-              ConverterUtils.toContainerId(entry.key.id);
+              ContainerId.fromString(entry.key.id);
           if (containerId.getApplicationAttemptId().equals(appAttemptId)) {
             ContainerHistoryData historyData =
                 historyDataMap.get(containerId);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c77a1095/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/utils/BuilderUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/utils/BuilderUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/utils/BuilderUtils.java
index 22e45fa..134f22d 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/utils/BuilderUtils.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/utils/BuilderUtils.java
@@ -113,7 +113,7 @@ public class BuilderUtils {
   public static LocalResource newLocalResource(URI uri,
       LocalResourceType type, LocalResourceVisibility visibility, long size,
       long timestamp, boolean shouldBeUploadedToSharedCache) {
-    return newLocalResource(ConverterUtils.getYarnUrlFromURI(uri), type,
+    return newLocalResource(URL.fromURI(uri), type,
         visibility, size, timestamp, shouldBeUploadedToSharedCache);
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c77a1095/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/AppAttemptBlock.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/AppAttemptBlock.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/AppAttemptBlock.java
index 9c2a1ae..798c372 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/AppAttemptBlock.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/AppAttemptBlock.java
@@ -65,7 +65,7 @@ public class AppAttemptBlock extends HtmlBlock {
     }
 
     try {
-      appAttemptId = ConverterUtils.toApplicationAttemptId(attemptid);
+      appAttemptId = ApplicationAttemptId.fromString(attemptid);
     } catch (IllegalArgumentException e) {
       puts("Invalid application attempt ID: " + attemptid);
       return;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c77a1095/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/ContainerBlock.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/ContainerBlock.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/ContainerBlock.java
index cae8d2e..893e823 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/ContainerBlock.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/ContainerBlock.java
@@ -59,7 +59,7 @@ public class ContainerBlock extends HtmlBlock {
 
     ContainerId containerId = null;
     try {
-      containerId = ConverterUtils.toContainerId(containerid);
+      containerId = ContainerId.fromString(containerid);
     } catch (IllegalArgumentException e) {
       puts("Invalid container ID: " + containerid);
       return;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c77a1095/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/WebServices.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/WebServices.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/WebServices.java
index 19ea301..904c511 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/WebServices.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/WebServices.java
@@ -431,7 +431,7 @@ public class WebServices {
     }
     ApplicationId aid = null;
     try {
-      aid = ConverterUtils.toApplicationId(appId);
+      aid = ApplicationId.fromString(appId);
     } catch (Exception e) {
       throw new BadRequestException(e);
     }
@@ -449,7 +449,7 @@ public class WebServices {
     }
     ApplicationAttemptId aaid = null;
     try {
-      aaid = ConverterUtils.toApplicationAttemptId(appAttemptId);
+      aaid = ApplicationAttemptId.fromString(appAttemptId);
     } catch (Exception e) {
       throw new BadRequestException(e);
     }
@@ -466,7 +466,7 @@ public class WebServices {
     }
     ContainerId cid = null;
     try {
-      cid = ConverterUtils.toContainerId(containerId);
+      cid = ContainerId.fromString(containerId);
     } catch (Exception e) {
       throw new BadRequestException(e);
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c77a1095/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/DefaultContainerExecutor.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/DefaultContainerExecutor.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/DefaultContainerExecutor.java
index 8f4b122..f8f19bd 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/DefaultContainerExecutor.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/DefaultContainerExecutor.java
@@ -167,11 +167,10 @@ public class DefaultContainerExecutor extends ContainerExecutor {
     ContainerId containerId = container.getContainerId();
 
     // create container dirs on all disks
-    String containerIdStr = ConverterUtils.toString(containerId);
+    String containerIdStr = containerId.toString();
     String appIdStr =
-        ConverterUtils.toString(
             containerId.getApplicationAttemptId().
-                getApplicationId());
+                getApplicationId().toString();
     for (String sLocalDir : localDirs) {
       Path usersdir = new Path(sLocalDir, ContainerLocalizer.USERCACHE);
       Path userdir = new Path(usersdir, user);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c77a1095/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/DockerContainerExecutor.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/DockerContainerExecutor.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/DockerContainerExecutor.java
index 72da236..2b18469 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/DockerContainerExecutor.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/DockerContainerExecutor.java
@@ -195,9 +195,9 @@ public class DockerContainerExecutor extends ContainerExecutor {
     ContainerId containerId = container.getContainerId();
 
     // create container dirs on all disks
-    String containerIdStr = ConverterUtils.toString(containerId);
-    String appIdStr = ConverterUtils.toString(
-      containerId.getApplicationAttemptId().getApplicationId());
+    String containerIdStr = containerId.toString();
+    String appIdStr =
+        containerId.getApplicationAttemptId().getApplicationId().toString();
     for (String sLocalDir : localDirs) {
       Path usersdir = new Path(sLocalDir, ContainerLocalizer.USERCACHE);
       Path userdir = new Path(usersdir, userName);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c77a1095/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/LinuxContainerExecutor.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/LinuxContainerExecutor.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/LinuxContainerExecutor.java
index e46ce56..1072b5a 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/LinuxContainerExecutor.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/LinuxContainerExecutor.java
@@ -319,7 +319,7 @@ public class LinuxContainerExecutor extends ContainerExecutor {
     String runAsUser = getRunAsUser(user);
 
     ContainerId containerId = container.getContainerId();
-    String containerIdStr = ConverterUtils.toString(containerId);
+    String containerIdStr = containerId.toString();
 
     resourcesHandler.preExecute(containerId,
             container.getResource());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c77a1095/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/ContainerImpl.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/containermanager/container/ContainerImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/ContainerImpl.java
index 304488e..7a6e1cf 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/ContainerImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/ContainerImpl.java
@@ -1361,7 +1361,7 @@ public class ContainerImpl implements Container {
   public String toString() {
     this.readLock.lock();
     try {
-      return ConverterUtils.toString(this.containerId);
+      return this.containerId.toString();
     } finally {
       this.readLock.unlock();
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c77a1095/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/launcher/ContainerLaunch.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/containermanager/launcher/ContainerLaunch.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/launcher/ContainerLaunch.java
index a3b53e3..7e9030c 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/launcher/ContainerLaunch.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/launcher/ContainerLaunch.java
@@ -163,7 +163,7 @@ public class ContainerLaunch implements Callable<Integer> {
 
     final ContainerLaunchContext launchContext = container.getLaunchContext();
     ContainerId containerID = container.getContainerId();
-    String containerIdStr = ConverterUtils.toString(containerID);
+    String containerIdStr = containerID.toString();
     final List<String> command = launchContext.getCommands();
     int ret = -1;
 
@@ -326,7 +326,7 @@ public class ContainerLaunch implements Callable<Integer> {
   protected List<String> getContainerLogDirs(List<String> logDirs) {
     List<String> containerLogDirs = new ArrayList<>(logDirs.size());
     String appIdStr = app.getAppId().toString();
-    String containerIdStr = ConverterUtils.toString(container.getContainerId());
+    String containerIdStr = container.getContainerId().toString();
     String relativeContainerLogDir = ContainerLaunch
         .getRelativeContainerLogDir(appIdStr, containerIdStr);
 
@@ -520,7 +520,7 @@ public class ContainerLaunch implements Callable<Integer> {
   @SuppressWarnings("unchecked") // dispatcher not typed
   public void cleanupContainer() throws IOException {
     ContainerId containerId = container.getContainerId();
-    String containerIdStr = ConverterUtils.toString(containerId);
+    String containerIdStr = containerId.toString();
     LOG.info("Cleaning up container " + containerIdStr);
 
     try {
@@ -616,7 +616,7 @@ public class ContainerLaunch implements Callable<Integer> {
       throws IOException {
     ContainerId containerId =
         container.getContainerTokenIdentifier().getContainerID();
-    String containerIdStr = ConverterUtils.toString(containerId);
+    String containerIdStr = containerId.toString();
     String user = container.getUser();
     Signal signal = translateCommandToSignal(command);
     if (signal.equals(Signal.NULL)) {
@@ -708,7 +708,7 @@ public class ContainerLaunch implements Callable<Integer> {
    */
   private String getContainerPid(Path pidFilePath) throws Exception {
     String containerIdStr = 
-        ConverterUtils.toString(container.getContainerId());
+        container.getContainerId().toString();
     String processId = null;
     LOG.debug("Accessing pid for container " + containerIdStr
         + " from pid file " + pidFilePath);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c77a1095/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/launcher/ContainerRelaunch.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/containermanager/launcher/ContainerRelaunch.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/launcher/ContainerRelaunch.java
index 711d5cd..1292df6 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/launcher/ContainerRelaunch.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/launcher/ContainerRelaunch.java
@@ -65,7 +65,7 @@ public class ContainerRelaunch extends ContainerLaunch {
     }
 
     ContainerId containerId = container.getContainerId();
-    String containerIdStr = ConverterUtils.toString(containerId);
+    String containerIdStr = containerId.toString();
     int ret = -1;
     Path containerLogDir;
     try {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c77a1095/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/launcher/RecoveredContainerLaunch.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/containermanager/launcher/RecoveredContainerLaunch.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/launcher/RecoveredContainerLaunch.java
index b9bdcc6..3cd31b7 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/launcher/RecoveredContainerLaunch.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/launcher/RecoveredContainerLaunch.java
@@ -68,9 +68,9 @@ public class RecoveredContainerLaunch extends ContainerLaunch {
   public Integer call() {
     int retCode = ExitCode.LOST.getExitCode();
     ContainerId containerId = container.getContainerId();
-    String appIdStr = ConverterUtils.toString(
-        containerId.getApplicationAttemptId().getApplicationId());
-    String containerIdStr = ConverterUtils.toString(containerId);
+    String appIdStr =
+        containerId.getApplicationAttemptId().getApplicationId().toString();
+    String containerIdStr = containerId.toString();
 
     dispatcher.getEventHandler().handle(new ContainerEvent(containerId,
         ContainerEventType.CONTAINER_LAUNCHED));

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c77a1095/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/ContainerLocalizer.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/containermanager/localizer/ContainerLocalizer.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/ContainerLocalizer.java
index 57cc346..65fd9d8 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/ContainerLocalizer.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/ContainerLocalizer.java
@@ -56,6 +56,7 @@ import org.apache.hadoop.util.concurrent.HadoopExecutors;
 import org.apache.hadoop.yarn.YarnUncaughtExceptionHandler;
 import org.apache.hadoop.yarn.api.records.LocalResource;
 import org.apache.hadoop.yarn.api.records.SerializedException;
+import org.apache.hadoop.yarn.api.records.URL;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.factories.RecordFactory;
@@ -295,7 +296,7 @@ public class ContainerLocalizer {
         try {
           Path localPath = fPath.get();
           stat.setLocalPath(
-              ConverterUtils.getYarnUrlFromPath(localPath));
+              URL.fromPath(localPath));
           stat.setLocalSize(
               FileUtil.getDU(new File(localPath.getParent().toUri())));
           stat.setStatus(ResourceStatusType.FETCH_SUCCESS);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c77a1095/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/LocalResourceRequest.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/containermanager/localizer/LocalResourceRequest.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/LocalResourceRequest.java
index 607d0b4..d2e8e22 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/LocalResourceRequest.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/LocalResourceRequest.java
@@ -43,7 +43,7 @@ public class LocalResourceRequest
    */
   public LocalResourceRequest(LocalResource resource)
       throws URISyntaxException {
-    this(ConverterUtils.getPathFromYarnURL(resource.getResource()),
+    this(resource.getResource().toPath(),
         resource.getTimestamp(),
         resource.getType(),
         resource.getVisibility(),
@@ -133,7 +133,7 @@ public class LocalResourceRequest
 
   @Override
   public URL getResource() {
-    return ConverterUtils.getYarnUrlFromPath(loc);
+    return URL.fromPath(loc);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c77a1095/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/ResourceLocalizationService.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/containermanager/localizer/ResourceLocalizationService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/ResourceLocalizationService.java
index b2413ad..409cc29 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/ResourceLocalizationService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/ResourceLocalizationService.java
@@ -79,6 +79,7 @@ import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.LocalResource;
 import org.apache.hadoop.yarn.api.records.LocalResourceVisibility;
+import org.apache.hadoop.yarn.api.records.URL;
 import org.apache.hadoop.yarn.api.records.impl.pb.LocalResourcePBImpl;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.event.Dispatcher;
@@ -301,7 +302,7 @@ public class ResourceLocalizationService extends CompositeService
         trackerState = appEntry.getValue();
         if (!trackerState.isEmpty()) {
           ApplicationId appId = appEntry.getKey();
-          String appIdStr = ConverterUtils.toString(appId);
+          String appIdStr = appId.toString();
           LocalResourcesTracker tracker = new LocalResourcesTrackerImpl(user,
               appId, dispatcher, false, super.getConfig(), stateStore);
           LocalResourcesTracker oldTracker = appRsrc.putIfAbsent(appIdStr,
@@ -442,7 +443,7 @@ public class ResourceLocalizationService extends CompositeService
     String userName = app.getUser();
     privateRsrc.putIfAbsent(userName, new LocalResourcesTrackerImpl(userName,
         null, dispatcher, true, super.getConfig(), stateStore));
-    String appIdStr = ConverterUtils.toString(app.getAppId());
+    String appIdStr = app.getAppId().toString();
     appRsrc.putIfAbsent(appIdStr, new LocalResourcesTrackerImpl(app.getUser(),
         app.getAppId(), dispatcher, false, super.getConfig(), stateStore));
     // 1) Signal container init
@@ -491,7 +492,7 @@ public class ResourceLocalizationService extends CompositeService
   private void handleContainerResourcesLocalized(
       ContainerLocalizationEvent event) {
     Container c = event.getContainer();
-    String locId = ConverterUtils.toString(c.getContainerId());
+    String locId = c.getContainerId().toString();
     localizerTracker.endContainerLocalization(locId);
   }
 
@@ -528,14 +529,15 @@ public class ResourceLocalizationService extends CompositeService
             c.getContainerId()));
       }
     }
-    String locId = ConverterUtils.toString(c.getContainerId());
+    String locId = c.getContainerId().toString();
     localizerTracker.cleanupPrivLocalizers(locId);
     
     // Delete the container directories
     String userName = c.getUser();
     String containerIDStr = c.toString();
-    String appIDStr = ConverterUtils.toString(
-        c.getContainerId().getApplicationAttemptId().getApplicationId());
+    String appIDStr =
+        c.getContainerId().getApplicationAttemptId().getApplicationId()
+            .toString();
     
     // Try deleting from good local dirs and full local dirs because a dir might
     // have gone bad while the app was running(disk full). In addition
@@ -583,7 +585,7 @@ public class ResourceLocalizationService extends CompositeService
     ApplicationId appId = application.getAppId();
     String appIDStr = application.toString();
     LocalResourcesTracker appLocalRsrcsTracker =
-      appRsrc.remove(ConverterUtils.toString(appId));
+      appRsrc.remove(appId.toString());
     if (appLocalRsrcsTracker != null) {
       for (LocalizedResource rsrc : appLocalRsrcsTracker ) {
         Path localPath = rsrc.getLocalPath();
@@ -637,7 +639,7 @@ public class ResourceLocalizationService extends CompositeService
       case PRIVATE:
         return privateRsrc.get(user);
       case APPLICATION:
-        return appRsrc.get(ConverterUtils.toString(appId));
+        return appRsrc.get(appId.toString());
     }
   }
 
@@ -977,7 +979,7 @@ public class ResourceLocalizationService extends CompositeService
              LocalResourceRequest nextRsrc = nRsrc.getRequest();
              LocalResource next =
                  recordFactory.newRecordInstance(LocalResource.class);
-             next.setResource(ConverterUtils.getYarnUrlFromPath(nextRsrc
+             next.setResource(URL.fromPath(nextRsrc
                .getPath()));
              next.setTimestamp(nextRsrc.getTimestamp());
              next.setType(nextRsrc.getType());
@@ -1028,8 +1030,8 @@ public class ResourceLocalizationService extends CompositeService
             try {
             getLocalResourcesTracker(req.getVisibility(), user, applicationId)
               .handle(
-                new ResourceLocalizedEvent(req, ConverterUtils
-                  .getPathFromYarnURL(stat.getLocalPath()), stat.getLocalSize()));
+                new ResourceLocalizedEvent(req, stat.getLocalPath().toPath(),
+                    stat.getLocalSize()));
             } catch (URISyntaxException e) { }
 
             // unlocking the resource and removing it from scheduled resource
@@ -1142,8 +1144,8 @@ public class ResourceLocalizationService extends CompositeService
               .setNmPrivateContainerTokens(nmPrivateCTokensPath)
               .setNmAddr(localizationServerAddress)
               .setUser(context.getUser())
-              .setAppId(ConverterUtils.toString(context.getContainerId()
-                  .getApplicationAttemptId().getApplicationId()))
+              .setAppId(context.getContainerId()
+                  .getApplicationAttemptId().getApplicationId().toString())
               .setLocId(localizerId)
               .setDirsHandler(dirsHandler)
               .build());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c77a1095/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/event/LocalizerResourceRequestEvent.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/containermanager/localizer/event/LocalizerResourceRequestEvent.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/event/LocalizerResourceRequestEvent.java
index 2e05dd7..0e732a7 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/event/LocalizerResourceRequestEvent.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/event/LocalizerResourceRequestEvent.java
@@ -37,7 +37,7 @@ public class LocalizerResourceRequestEvent extends LocalizerEvent {
   public LocalizerResourceRequestEvent(LocalizedResource resource,
       LocalResourceVisibility vis, LocalizerContext context, String pattern) {
     super(LocalizerEventType.REQUEST_RESOURCE_LOCALIZATION,
-        ConverterUtils.toString(context.getContainerId()));
+        context.getContainerId().toString());
     this.vis = vis;
     this.context = context;
     this.resource = resource;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c77a1095/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/sharedcache/SharedCacheUploader.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/containermanager/localizer/sharedcache/SharedCacheUploader.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/sharedcache/SharedCacheUploader.java
index 682b272..b034e7a 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/sharedcache/SharedCacheUploader.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/sharedcache/SharedCacheUploader.java
@@ -211,7 +211,7 @@ class SharedCacheUploader implements Callable<Boolean> {
 
     final Path remotePath;
     try {
-      remotePath = ConverterUtils.getPathFromYarnURL(resource.getResource());
+      remotePath = resource.getResource().toPath();
     } catch (URISyntaxException e) {
       throw new IOException("Invalid resource", e);
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c77a1095/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/logaggregation/AppLogAggregatorImpl.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/containermanager/logaggregation/AppLogAggregatorImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/logaggregation/AppLogAggregatorImpl.java
index ba7836a..c70fa5b 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/logaggregation/AppLogAggregatorImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/logaggregation/AppLogAggregatorImpl.java
@@ -164,7 +164,7 @@ public class AppLogAggregatorImpl implements AppLogAggregator {
     this.conf = conf;
     this.delService = deletionService;
     this.appId = appId;
-    this.applicationId = ConverterUtils.toString(appId);
+    this.applicationId = appId.toString();
     this.userUgi = userUgi;
     this.dirsHandler = dirsHandler;
     this.remoteNodeLogFileForApp = remoteNodeLogFileForApp;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c77a1095/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/recovery/NMLeveldbStateStoreService.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/recovery/NMLeveldbStateStoreService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/recovery/NMLeveldbStateStoreService.java
index 8bd2040..5fe2713 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/recovery/NMLeveldbStateStoreService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/recovery/NMLeveldbStateStoreService.java
@@ -186,7 +186,7 @@ public class NMLeveldbStateStoreService extends NMStateStoreService {
         if (idEndPos < 0) {
           throw new IOException("Unable to determine container in key: " + key);
         }
-        ContainerId containerId = ConverterUtils.toContainerId(
+        ContainerId containerId = ContainerId.fromString(
             key.substring(CONTAINERS_KEY_PREFIX.length(), idEndPos));
         String keyPrefix = key.substring(0, idEndPos+1);
         RecoveredContainerState rcs = loadContainerState(containerId,
@@ -654,7 +654,7 @@ public class NMLeveldbStateStoreService extends NMStateStoreService {
           throw new IOException("Unable to determine appID in resource key: "
               + key);
         }
-        ApplicationId appId = ConverterUtils.toApplicationId(
+        ApplicationId appId = ApplicationId.fromString(
             key.substring(appIdStartPos, appIdEndPos));
         userResources.appTrackerStates.put(appId,
             loadResourceTrackerState(iter, key.substring(0, appIdEndPos+1)));
@@ -822,7 +822,7 @@ public class NMLeveldbStateStoreService extends NMStateStoreService {
             ApplicationAttemptId.appAttemptIdStrPrefix)) {
           ApplicationAttemptId attempt;
           try {
-            attempt = ConverterUtils.toApplicationAttemptId(key);
+            attempt = ApplicationAttemptId.fromString(key);
           } catch (IllegalArgumentException e) {
             throw new IOException("Bad application master key state for "
                 + fullKey, e);
@@ -926,7 +926,7 @@ public class NMLeveldbStateStoreService extends NMStateStoreService {
     ContainerId containerId;
     Long expTime;
     try {
-      containerId = ConverterUtils.toContainerId(containerIdStr);
+      containerId = ContainerId.fromString(containerIdStr);
       expTime = Long.parseLong(asString(value));
     } catch (IllegalArgumentException e) {
       throw new IOException("Bad container token state for " + key, e);
@@ -988,7 +988,7 @@ public class NMLeveldbStateStoreService extends NMStateStoreService {
         String appIdStr = fullKey.substring(logDeleterKeyPrefixLength);
         ApplicationId appId = null;
         try {
-          appId = ConverterUtils.toApplicationId(appIdStr);
+          appId = ApplicationId.fromString(appIdStr);
         } catch (IllegalArgumentException e) {
           LOG.warn("Skipping unknown log deleter key " + fullKey);
           continue;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c77a1095/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/util/NodeManagerBuilderUtils.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/util/NodeManagerBuilderUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/util/NodeManagerBuilderUtils.java
index 21cf1f2..21c3c06 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/util/NodeManagerBuilderUtils.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/util/NodeManagerBuilderUtils.java
@@ -28,7 +28,7 @@ public class NodeManagerBuilderUtils {
 
   public static ResourceLocalizationSpec newResourceLocalizationSpec(
       LocalResource rsrc, Path path) {
-    URL local = ConverterUtils.getYarnUrlFromPath(path);
+    URL local = URL.fromPath(path);
     ResourceLocalizationSpec resourceLocalizationSpec =
         Records.newRecord(ResourceLocalizationSpec.class);
     resourceLocalizationSpec.setDestinationDirectory(local);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c77a1095/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/util/ProcessIdFileReader.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/util/ProcessIdFileReader.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/util/ProcessIdFileReader.java
index 80d4db2..5a7dba7 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/util/ProcessIdFileReader.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/util/ProcessIdFileReader.java
@@ -27,6 +27,7 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.util.Shell;
+import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.util.ConverterUtils;
 
 /**
@@ -69,7 +70,7 @@ public class ProcessIdFileReader {
               // On Windows, pid is expected to be a container ID, so find first
               // line that parses successfully as a container ID.
               try {
-                ConverterUtils.toContainerId(temp);
+                ContainerId.fromString(temp);
                 processId = temp;
                 break;
               } catch (Exception e) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c77a1095/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/ApplicationPage.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/webapp/ApplicationPage.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/ApplicationPage.java
index bc90d8e..2783b18 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/ApplicationPage.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/ApplicationPage.java
@@ -76,10 +76,9 @@ public class ApplicationPage extends NMView implements YarnWebParams {
 
     @Override
     protected void render(Block html) {
-      ApplicationId applicationID = null;
+      ApplicationId applicationID;
       try {
-        applicationID = ConverterUtils.toApplicationId(this.recordFactory,
-            $(APPLICATION_ID));
+        applicationID = ApplicationId.fromString($(APPLICATION_ID));
       } catch (IllegalArgumentException e) {
         html.p()._("Invalid Application Id " + $(APPLICATION_ID))._();
         return;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c77a1095/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/ContainerLogsPage.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/webapp/ContainerLogsPage.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/ContainerLogsPage.java
index 2fd6b2c..3e5f4d2 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/ContainerLogsPage.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/ContainerLogsPage.java
@@ -92,7 +92,7 @@ public class ContainerLogsPage extends NMView {
 
       ContainerId containerId;
       try {
-        containerId = ConverterUtils.toContainerId($(CONTAINER_ID));
+        containerId = ContainerId.fromString($(CONTAINER_ID));
       } catch (IllegalArgumentException ex) {
         html.h1("Invalid container ID: " + $(CONTAINER_ID));
         return;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c77a1095/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/ContainerLogsUtils.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/webapp/ContainerLogsUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/ContainerLogsUtils.java
index 319f49b..35e7593 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/ContainerLogsUtils.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/ContainerLogsUtils.java
@@ -78,8 +78,8 @@ public class ContainerLogsUtils {
     List<File> containerLogDirs = new ArrayList<File>(logDirs.size());
     for (String logDir : logDirs) {
       logDir = new File(logDir).toURI().getPath();
-      String appIdStr = ConverterUtils.toString(containerId
-          .getApplicationAttemptId().getApplicationId());
+      String appIdStr = containerId
+          .getApplicationAttemptId().getApplicationId().toString();
       File appLogDir = new File(logDir, appIdStr);
       containerLogDirs.add(new File(appLogDir, containerId.toString()));
     }
@@ -160,7 +160,7 @@ public class ContainerLogsUtils {
   
   public static FileInputStream openLogFileForRead(String containerIdStr, File logFile,
       Context context) throws IOException {
-    ContainerId containerId = ConverterUtils.toContainerId(containerIdStr);
+    ContainerId containerId = ContainerId.fromString(containerIdStr);
     ApplicationId applicationId = containerId.getApplicationAttemptId()
         .getApplicationId();
     String user = context.getApplications().get(

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c77a1095/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/ContainerPage.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/webapp/ContainerPage.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/ContainerPage.java
index f4367bc..a1e0bc7 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/ContainerPage.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/ContainerPage.java
@@ -63,7 +63,7 @@ public class ContainerPage extends NMView implements YarnWebParams {
     protected void render(Block html) {
       ContainerId containerID;
       try {
-        containerID = ConverterUtils.toContainerId($(CONTAINER_ID));
+        containerID = ContainerId.fromString($(CONTAINER_ID));
       } catch (IllegalArgumentException e) {
         html.p()._("Invalid containerId " + $(CONTAINER_ID))._();
         return;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c77a1095/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/NMWebServices.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/webapp/NMWebServices.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/NMWebServices.java
index efc0e7e..3a30392 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/NMWebServices.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/NMWebServices.java
@@ -181,7 +181,7 @@ public class NMWebServices {
     ContainerId containerId = null;
     init();
     try {
-      containerId = ConverterUtils.toContainerId(id);
+      containerId = ContainerId.fromString(id);
     } catch (Exception e) {
       throw new BadRequestException("invalid container id, " + id);
     }
@@ -224,7 +224,7 @@ public class NMWebServices {
       @QueryParam("size") String size) {
     ContainerId containerId;
     try {
-      containerId = ConverterUtils.toContainerId(containerIdStr);
+      containerId = ContainerId.fromString(containerIdStr);
     } catch (IllegalArgumentException ex) {
       return Response.status(Status.BAD_REQUEST).build();
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c77a1095/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/dao/AppInfo.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/webapp/dao/AppInfo.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/dao/AppInfo.java
index 95e2a65..f55ca81 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/dao/AppInfo.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/dao/AppInfo.java
@@ -42,14 +42,14 @@ public class AppInfo {
   } // JAXB needs this
 
   public AppInfo(final Application app) {
-    this.id = ConverterUtils.toString(app.getAppId());
+    this.id = app.getAppId().toString();
     this.state = app.getApplicationState().toString();
     this.user = app.getUser();
 
     this.containerids = new ArrayList<String>();
     Map<ContainerId, Container> appContainers = app.getContainers();
     for (ContainerId containerId : appContainers.keySet()) {
-      String containerIdStr = ConverterUtils.toString(containerId);
+      String containerIdStr = containerId.toString();
       containerids.add(containerIdStr);
     }
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c77a1095/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestNodeManagerReboot.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestNodeManagerReboot.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestNodeManagerReboot.java
index 9fb8ebf..5f9b883 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestNodeManagerReboot.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestNodeManagerReboot.java
@@ -118,7 +118,7 @@ public class TestNodeManagerReboot {
     ContainerId cId = createContainerId();
 
     URL localResourceUri =
-        ConverterUtils.getYarnUrlFromPath(localFS.makeQualified(new Path(
+        URL.fromPath(localFS.makeQualified(new Path(
           localResourceDir.getAbsolutePath())));
 
     LocalResource localResource =

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c77a1095/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestNodeManagerResync.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestNodeManagerResync.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestNodeManagerResync.java
index b3d44f5..ee2677c 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestNodeManagerResync.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestNodeManagerResync.java
@@ -741,7 +741,7 @@ public class TestNodeManagerResync {
       ContainerLaunchContext containerLaunchContext =
           recordFactory.newRecordInstance(ContainerLaunchContext.class);
       URL resource_alpha =
-          ConverterUtils.getYarnUrlFromPath(localFS
+          URL.fromPath(localFS
               .makeQualified(new Path(scriptFile.getAbsolutePath())));
       LocalResource rsrc_alpha =
           recordFactory.newRecordInstance(LocalResource.class);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c77a1095/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestNodeManagerShutdown.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestNodeManagerShutdown.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestNodeManagerShutdown.java
index 980c764..b3ad318 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestNodeManagerShutdown.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestNodeManagerShutdown.java
@@ -200,7 +200,7 @@ public class TestNodeManagerShutdown {
         .getCanonicalHostName(), port);
     
     URL localResourceUri =
-        ConverterUtils.getYarnUrlFromPath(localFS
+        URL.fromPath(localFS
             .makeQualified(new Path(scriptFile.getAbsolutePath())));
     LocalResource localResource =
         recordFactory.newRecordInstance(LocalResource.class);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c77a1095/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/api/protocolrecords/impl/pb/TestPBRecordImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/api/protocolrecords/impl/pb/TestPBRecordImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/api/protocolrecords/impl/pb/TestPBRecordImpl.java
index 03937ea..6ceaa75 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/api/protocolrecords/impl/pb/TestPBRecordImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/api/protocolrecords/impl/pb/TestPBRecordImpl.java
@@ -25,6 +25,7 @@ import java.io.IOException;
 import java.net.URISyntaxException;
 import java.util.ArrayList;
 
+import org.apache.hadoop.yarn.api.records.URL;
 import org.junit.Assert;
 
 import org.apache.hadoop.conf.Configuration;
@@ -62,7 +63,7 @@ public class TestPBRecordImpl {
   static LocalResource createResource() {
     LocalResource ret = recordFactory.newRecordInstance(LocalResource.class);
     assertTrue(ret instanceof LocalResourcePBImpl);
-    ret.setResource(ConverterUtils.getYarnUrlFromPath(new Path(
+    ret.setResource(URL.fromPath(new Path(
       "hdfs://y.ak:9820/foo/bar")));
     ret.setSize(4344L);
     ret.setTimestamp(3141592653589793L);
@@ -76,7 +77,7 @@ public class TestPBRecordImpl {
     assertTrue(ret instanceof LocalResourceStatusPBImpl);
     ret.setResource(createResource());
     ret.setLocalPath(
-        ConverterUtils.getYarnUrlFromPath(
+        URL.fromPath(
           new Path("file:///local/foo/bar")));
     ret.setStatus(ResourceStatusType.FETCH_SUCCESS);
     ret.setLocalSize(4443L);
@@ -109,8 +110,8 @@ public class TestPBRecordImpl {
     ResourceLocalizationSpec resource =
       recordFactory.newRecordInstance(ResourceLocalizationSpec.class);
     resource.setResource(rsrc);
-    resource.setDestinationDirectory(ConverterUtils
-      .getYarnUrlFromPath(new Path("/tmp" + System.currentTimeMillis())));
+    resource.setDestinationDirectory(
+        URL.fromPath((new Path("/tmp" + System.currentTimeMillis()))));
     rsrcs.add(resource);
     ret.setResourceSpecs(rsrcs);
     System.out.println(resource);


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


[28/39] hadoop git commit: HADOOP-13189. FairCallQueue makes callQueue larger than the configured capacity. Contributed by Vinitha Gankidi.

Posted by ae...@apache.org.
HADOOP-13189. FairCallQueue makes callQueue larger than the configured capacity. Contributed by Vinitha Gankidi.

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

Branch: refs/heads/HDFS-1312
Commit: a2a5cb60b09491cb672978ba9442f02373392c67
Parents: bf78040
Author: Konstantin V Shvachko <sh...@apache.org>
Authored: Thu Jun 16 18:20:49 2016 -0700
Committer: Konstantin V Shvachko <sh...@apache.org>
Committed: Thu Jun 16 18:20:49 2016 -0700

----------------------------------------------------------------------
 .../org/apache/hadoop/ipc/CallQueueManager.java |  4 +--
 .../org/apache/hadoop/ipc/FairCallQueue.java    | 19 +++++++++-----
 .../apache/hadoop/ipc/TestCallQueueManager.java |  4 +--
 .../apache/hadoop/ipc/TestFairCallQueue.java    | 27 +++++++++++++++-----
 4 files changed, 37 insertions(+), 17 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/a2a5cb60/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/CallQueueManager.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/CallQueueManager.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/CallQueueManager.java
index 7a19217..cbf8ebd 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/CallQueueManager.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/CallQueueManager.java
@@ -72,8 +72,8 @@ public class CallQueueManager<E> {
     this.clientBackOffEnabled = clientBackOffEnabled;
     this.putRef = new AtomicReference<BlockingQueue<E>>(bq);
     this.takeRef = new AtomicReference<BlockingQueue<E>>(bq);
-    LOG.info("Using callQueue: " + backingClass + " scheduler: " +
-        schedulerClass);
+    LOG.info("Using callQueue: " + backingClass + " queueCapacity: " +
+        maxQueueSize + " scheduler: " + schedulerClass);
   }
 
   private static <T extends RpcScheduler> T createScheduler(

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a2a5cb60/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/FairCallQueue.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/FairCallQueue.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/FairCallQueue.java
index 435c454..38b196d 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/FairCallQueue.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/FairCallQueue.java
@@ -75,11 +75,12 @@ public class FairCallQueue<E extends Schedulable> extends AbstractQueue<E>
 
   /**
    * Create a FairCallQueue.
-   * @param capacity the maximum size of each sub-queue
+   * @param capacity the total size of all sub-queues
    * @param ns the prefix to use for configuration
    * @param conf the configuration to read from
-   * Notes: the FairCallQueue has no fixed capacity. Rather, it has a minimum
-   * capacity of `capacity` and a maximum capacity of `capacity * number_queues`
+   * Notes: Each sub-queue has a capacity of `capacity / numSubqueues`.
+   * The first or the highest priority sub-queue has an excess capacity
+   * of `capacity % numSubqueues`
    */
   public FairCallQueue(int priorityLevels, int capacity, String ns,
       Configuration conf) {
@@ -88,13 +89,19 @@ public class FairCallQueue<E extends Schedulable> extends AbstractQueue<E>
           "at least 1");
     }
     int numQueues = priorityLevels;
-    LOG.info("FairCallQueue is in use with " + numQueues + " queues.");
+    LOG.info("FairCallQueue is in use with " + numQueues +
+        " queues with total capacity of " + capacity);
 
     this.queues = new ArrayList<BlockingQueue<E>>(numQueues);
     this.overflowedCalls = new ArrayList<AtomicLong>(numQueues);
-
+    int queueCapacity = capacity / numQueues;
+    int capacityForFirstQueue = queueCapacity + (capacity % numQueues);
     for(int i=0; i < numQueues; i++) {
-      this.queues.add(new LinkedBlockingQueue<E>(capacity));
+      if (i == 0) {
+        this.queues.add(new LinkedBlockingQueue<E>(capacityForFirstQueue));
+      } else {
+        this.queues.add(new LinkedBlockingQueue<E>(queueCapacity));
+      }
       this.overflowedCalls.add(new AtomicLong(0));
     }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a2a5cb60/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestCallQueueManager.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestCallQueueManager.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestCallQueueManager.java
index af9ce1b..1211657 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestCallQueueManager.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestCallQueueManager.java
@@ -214,9 +214,9 @@ public class TestCallQueueManager {
     assertTrue(queue.getCanonicalName().equals(queueClassName));
 
     manager = new CallQueueManager<FakeCall>(queue, scheduler, false,
-        2, "", conf);
+        8, "", conf);
 
-    // Default FCQ has 4 levels and the max capacity is 2 x 4
+    // Default FCQ has 4 levels and the max capacity is 8
     assertCanPut(manager, 3, 3);
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a2a5cb60/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestFairCallQueue.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestFairCallQueue.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestFairCallQueue.java
index 4a8ad3b..8c96c2e 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestFairCallQueue.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestFairCallQueue.java
@@ -18,12 +18,6 @@
 
 package org.apache.hadoop.ipc;
 
-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.mockito.Mockito.mock;
 import static org.mockito.Mockito.when;
 
@@ -68,7 +62,26 @@ public class TestFairCallQueue extends TestCase {
     Configuration conf = new Configuration();
     conf.setInt("ns." + FairCallQueue.IPC_CALLQUEUE_PRIORITY_LEVELS_KEY, 2);
 
-    fcq = new FairCallQueue<Schedulable>(2, 5, "ns", conf);
+    fcq = new FairCallQueue<Schedulable>(2, 10, "ns", conf);
+  }
+
+  // Validate that the total capacity of all subqueues equals
+  // the maxQueueSize for different values of maxQueueSize
+  public void testTotalCapacityOfSubQueues() {
+    Configuration conf = new Configuration();
+    FairCallQueue<Schedulable> fairCallQueue;
+    fairCallQueue = new FairCallQueue<Schedulable>(1, 1000, "ns", conf);
+    assertEquals(fairCallQueue.remainingCapacity(), 1000);
+    fairCallQueue = new FairCallQueue<Schedulable>(4, 1000, "ns", conf);
+    assertEquals(fairCallQueue.remainingCapacity(), 1000);
+    fairCallQueue = new FairCallQueue<Schedulable>(7, 1000, "ns", conf);
+    assertEquals(fairCallQueue.remainingCapacity(), 1000);
+    fairCallQueue = new FairCallQueue<Schedulable>(1, 1025, "ns", conf);
+    assertEquals(fairCallQueue.remainingCapacity(), 1025);
+    fairCallQueue = new FairCallQueue<Schedulable>(4, 1025, "ns", conf);
+    assertEquals(fairCallQueue.remainingCapacity(), 1025);
+    fairCallQueue = new FairCallQueue<Schedulable>(7, 1025, "ns", conf);
+    assertEquals(fairCallQueue.remainingCapacity(), 1025);
   }
 
   //


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


[29/39] hadoop git commit: MAPREDUCE-6542. HistoryViewer uses SimpleDateFormat, but SimpleDateFormat is not threadsafe. Contributed by zhangyubiao.

Posted by ae...@apache.org.
MAPREDUCE-6542. HistoryViewer uses SimpleDateFormat, but SimpleDateFormat is not threadsafe. Contributed by zhangyubiao.


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

Branch: refs/heads/HDFS-1312
Commit: 51d497fa93d35dec7503eb30b8e3a1e8f2d39b45
Parents: a2a5cb6
Author: Akira Ajisaka <aa...@apache.org>
Authored: Fri Jun 17 11:03:54 2016 +0900
Committer: Akira Ajisaka <aa...@apache.org>
Committed: Fri Jun 17 11:03:54 2016 +0900

----------------------------------------------------------------------
 .../org/apache/hadoop/util/StringUtils.java     | 54 ++++++++++++++++----
 .../org/apache/hadoop/util/TestStringUtils.java | 44 ++++++++++++++--
 .../HumanReadableHistoryViewerPrinter.java      | 15 +++---
 3 files changed, 93 insertions(+), 20 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/51d497fa/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/StringUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/StringUtils.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/StringUtils.java
index 31f6914..e8d76a3 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/StringUtils.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/StringUtils.java
@@ -26,7 +26,6 @@ import java.text.DateFormat;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
-import java.util.Date;
 import java.util.Iterator;
 import java.util.LinkedHashSet;
 import java.util.List;
@@ -38,6 +37,7 @@ import java.util.regex.Matcher;
 import java.util.regex.Pattern;
 
 import org.apache.commons.lang.SystemUtils;
+import org.apache.commons.lang.time.FastDateFormat;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.fs.Path;
@@ -323,20 +323,56 @@ public class StringUtils {
   }
 
   /**
-   * Formats time in ms and appends difference (finishTime - startTime) 
+   * @param dateFormat date format to use
+   * @param finishTime finish time
+   * @param startTime  start time
+   * @return formatted value.
+   * Formats time in ms and appends difference (finishTime - startTime)
+   * as returned by formatTimeDiff().
+   * If finish time is 0, empty string is returned, if start time is 0
+   * then difference is not appended to return value.
+   * @deprecated Use
+   * {@link StringUtils#getFormattedTimeWithDiff(FastDateFormat, long, long)} or
+   * {@link StringUtils#getFormattedTimeWithDiff(String, long, long)} instead.
+   */
+  @Deprecated
+  public static String getFormattedTimeWithDiff(DateFormat dateFormat,
+      long finishTime, long startTime){
+    String formattedFinishTime = dateFormat.format(finishTime);
+    return getFormattedTimeWithDiff(formattedFinishTime, finishTime, startTime);
+  }
+
+  /**
+   * Formats time in ms and appends difference (finishTime - startTime)
    * as returned by formatTimeDiff().
-   * If finish time is 0, empty string is returned, if start time is 0 
-   * then difference is not appended to return value. 
+   * If finish time is 0, empty string is returned, if start time is 0
+   * then difference is not appended to return value.
+   *
    * @param dateFormat date format to use
-   * @param finishTime fnish time
+   * @param finishTime finish time
+   * @param startTime  start time
+   * @return formatted value.
+   */
+  public static String getFormattedTimeWithDiff(FastDateFormat dateFormat,
+      long finishTime, long startTime) {
+    String formattedFinishTime = dateFormat.format(finishTime);
+    return getFormattedTimeWithDiff(formattedFinishTime, finishTime, startTime);
+  }
+  /**
+   * Formats time in ms and appends difference (finishTime - startTime)
+   * as returned by formatTimeDiff().
+   * If finish time is 0, empty string is returned, if start time is 0
+   * then difference is not appended to return value.
+   * @param formattedFinishTime formattedFinishTime to use
+   * @param finishTime finish time
    * @param startTime start time
-   * @return formatted value. 
+   * @return formatted value.
    */
-  public static String getFormattedTimeWithDiff(DateFormat dateFormat, 
-                                                long finishTime, long startTime){
+  public static String getFormattedTimeWithDiff(String formattedFinishTime,
+      long finishTime, long startTime){
     StringBuilder buf = new StringBuilder();
     if (0 != finishTime) {
-      buf.append(dateFormat.format(new Date(finishTime)));
+      buf.append(formattedFinishTime);
       if (0 != startTime){
         buf.append(" (" + formatTimeDiff(finishTime , startTime) + ")");
       }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/51d497fa/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestStringUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestStringUtils.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestStringUtils.java
index e3e613c..1f474f8 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestStringUtils.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestStringUtils.java
@@ -33,11 +33,16 @@ import java.util.Collection;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
+import java.util.concurrent.BrokenBarrierException;
+import java.util.concurrent.CyclicBarrier;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
 import java.util.regex.Pattern;
 
+import org.apache.commons.lang.time.FastDateFormat;
 import org.apache.hadoop.test.UnitTestcaseTimeLimit;
 import org.apache.hadoop.util.StringUtils.TraditionalBinaryPrefix;
-import org.junit.Assume;
 import org.junit.Test;
 
 public class TestStringUtils extends UnitTestcaseTimeLimit {
@@ -51,6 +56,9 @@ public class TestStringUtils extends UnitTestcaseTimeLimit {
   final private static String STR_WITH_BOTH2 = ",A\\,,B\\\\,";
   final private static String ESCAPED_STR_WITH_BOTH2 = 
     "\\,A\\\\\\,\\,B\\\\\\\\\\,";
+
+  final private static FastDateFormat FAST_DATE_FORMAT =
+      FastDateFormat.getInstance("d-MMM-yyyy HH:mm:ss");
   
   @Test (timeout = 30000)
   public void testEscapeString() throws Exception {
@@ -387,8 +395,6 @@ public class TestStringUtils extends UnitTestcaseTimeLimit {
       pattern, replacements));
     assertEquals("___", StringUtils.replaceTokens("$UNDER_SCORES", pattern,
       replacements));
-    assertEquals("//one//two//", StringUtils.replaceTokens("//$FOO/$BAR/$BAZ//",
-      pattern, replacements));
   }
 
   @Test (timeout = 5000)
@@ -438,6 +444,38 @@ public class TestStringUtils extends UnitTestcaseTimeLimit {
     }
   }
 
+  @Test
+  //Multithreaded Test GetFormattedTimeWithDiff()
+  public void testGetFormattedTimeWithDiff() throws InterruptedException {
+    ExecutorService executorService = Executors.newFixedThreadPool(16);
+    final CyclicBarrier cyclicBarrier = new CyclicBarrier(10);
+    for (int i = 0; i < 10; i++) {
+
+      executorService.execute(new Runnable() {
+        @Override
+        public void run() {
+          try {
+            cyclicBarrier.await();
+          } catch (InterruptedException | BrokenBarrierException e) {
+            //Ignored
+          }
+          final long end = System.currentTimeMillis();
+          final long start = end - 30000;
+          String formattedTime1 = StringUtils.getFormattedTimeWithDiff(
+              FAST_DATE_FORMAT, start, end);
+          String formattedTime2 = StringUtils.getFormattedTimeWithDiff(
+              FAST_DATE_FORMAT, start, end);
+          assertTrue("Method returned inconsistent results indicative of"
+              + " a race condition", formattedTime1.equals(formattedTime2));
+
+        }
+      });
+    }
+
+    executorService.shutdown();
+    executorService.awaitTermination(50, TimeUnit.SECONDS);
+  }
+
   // Benchmark for StringUtils split
   public static void main(String []args) {
     final String TO_SPLIT = "foo,bar,baz,blah,blah";

http://git-wip-us.apache.org/repos/asf/hadoop/blob/51d497fa/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/HumanReadableHistoryViewerPrinter.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/HumanReadableHistoryViewerPrinter.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/HumanReadableHistoryViewerPrinter.java
index 30903af..d3da9f4 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/HumanReadableHistoryViewerPrinter.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/HumanReadableHistoryViewerPrinter.java
@@ -17,6 +17,7 @@
  */
 package org.apache.hadoop.mapreduce.jobhistory;
 
+import org.apache.commons.lang.time.FastDateFormat;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.mapred.JobStatus;
@@ -32,7 +33,6 @@ import java.io.IOException;
 import java.io.PrintStream;
 import java.text.DecimalFormat;
 import java.text.Format;
-import java.text.SimpleDateFormat;
 import java.util.Arrays;
 import java.util.Comparator;
 import java.util.Iterator;
@@ -49,23 +49,22 @@ import java.util.TimeZone;
 class HumanReadableHistoryViewerPrinter implements HistoryViewerPrinter {
 
   private JobHistoryParser.JobInfo job;
-  private final SimpleDateFormat dateFormat;
+  private final FastDateFormat dateFormat;
   private boolean printAll;
   private String scheme;
 
   HumanReadableHistoryViewerPrinter(JobHistoryParser.JobInfo job,
                                     boolean printAll, String scheme) {
-    this.job = job;
-    this.printAll = printAll;
-    this.scheme = scheme;
-    this.dateFormat = new SimpleDateFormat("d-MMM-yyyy HH:mm:ss");
+    this(job, printAll, scheme, TimeZone.getDefault());
   }
 
   HumanReadableHistoryViewerPrinter(JobHistoryParser.JobInfo job,
                                     boolean printAll, String scheme,
                                     TimeZone tz) {
-    this(job, printAll, scheme);
-    this.dateFormat.setTimeZone(tz);
+    this.job = job;
+    this.printAll = printAll;
+    this.scheme = scheme;
+    this.dateFormat = FastDateFormat.getInstance("d-MMM-yyyy HH:mm:ss", tz);
   }
 
   /**


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


[22/39] hadoop git commit: HADOOP-12875. [Azure Data Lake] Support for contract test and unit test cases. Contributed by Vishwajeet Dusane.

Posted by ae...@apache.org.
HADOOP-12875. [Azure Data Lake] Support for contract test and unit test cases. Contributed by Vishwajeet Dusane.


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

Branch: refs/heads/HDFS-1312
Commit: c9e71382a58b6ffcb3fccb79d3c146877f1c8313
Parents: e14ee0d
Author: Chris Nauroth <cn...@apache.org>
Authored: Thu Jun 16 09:46:05 2016 -0700
Committer: Chris Nauroth <cn...@apache.org>
Committed: Thu Jun 16 09:46:05 2016 -0700

----------------------------------------------------------------------
 .../web/PrivateAzureDataLakeFileSystem.java     |   2 +-
 .../hadoop/fs/adl/TestADLResponseData.java      | 120 +++++++
 .../apache/hadoop/fs/adl/TestGetFileStatus.java |  65 ++++
 .../apache/hadoop/fs/adl/TestListStatus.java    |  95 ++++++
 .../hadoop/fs/adl/TestableAdlFileSystem.java    |  31 ++
 .../fs/adl/live/AdlStorageConfiguration.java    |  76 +++++
 .../hadoop/fs/adl/live/AdlStorageContract.java  |  65 ++++
 .../fs/adl/live/TestAdlContractAppendLive.java  |  53 +++
 .../fs/adl/live/TestAdlContractConcatLive.java  |  52 +++
 .../fs/adl/live/TestAdlContractCreateLive.java  |  52 +++
 .../fs/adl/live/TestAdlContractDeleteLive.java  |  44 +++
 .../fs/adl/live/TestAdlContractMkdirLive.java   |  55 +++
 .../fs/adl/live/TestAdlContractOpenLive.java    |  44 +++
 .../fs/adl/live/TestAdlContractRenameLive.java  |  63 ++++
 .../fs/adl/live/TestAdlContractRootDirLive.java |  52 +++
 .../fs/adl/live/TestAdlContractSeekLive.java    |  44 +++
 .../live/TestAdlDifferentSizeWritesLive.java    | 102 ++++++
 .../adl/live/TestAdlFileSystemContractLive.java | 111 ++++++
 .../hadoop/fs/adl/live/TestAdlReadLive.java     | 342 +++++++++++++++++++
 ...estAdlWebHdfsFileContextCreateMkdirLive.java |  79 +++++
 ...AdlWebHdfsFileContextMainOperationsLive.java | 104 ++++++
 ...hedRefreshTokenBasedAccessTokenProvider.java |   8 +-
 .../hadoop/fs/common/AdlMockWebServer.java      | 116 +++++++
 .../hadoop/fs/common/ExpectedResponse.java      |  72 ++++
 .../hadoop/fs/common/TestDataForRead.java       | 120 +++++++
 .../org/apache/hadoop/hdfs/web/TestAdlRead.java | 205 +++++++++++
 .../web/TestConcurrentDataReadOperations.java   | 306 +++++++++++++++++
 .../hdfs/web/TestConfigurationSetting.java      | 112 ++++++
 .../hdfs/web/TestSplitSizeCalculation.java      | 123 +++++++
 .../src/test/resources/adls.xml                 | 139 ++++++++
 .../test/resources/contract-test-options.xml    |  57 ++++
 31 files changed, 2905 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/c9e71382/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/hdfs/web/PrivateAzureDataLakeFileSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/hdfs/web/PrivateAzureDataLakeFileSystem.java b/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/hdfs/web/PrivateAzureDataLakeFileSystem.java
index 89011d2..c4a19d5 100644
--- a/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/hdfs/web/PrivateAzureDataLakeFileSystem.java
+++ b/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/hdfs/web/PrivateAzureDataLakeFileSystem.java
@@ -922,7 +922,7 @@ public class PrivateAzureDataLakeFileSystem extends SWebHdfsFileSystem {
         size = maxBufferSize;
       }
 
-      int equalBufferSplit = Math.max(Math.round(size / SIZE4MB), 1);
+      int equalBufferSplit = Math.max(size / SIZE4MB, 1);
       int splitSize = Math.min(equalBufferSplit, maxConcurrentConnection);
       return splitSize;
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c9e71382/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/TestADLResponseData.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/TestADLResponseData.java b/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/TestADLResponseData.java
new file mode 100644
index 0000000..1c4fcab
--- /dev/null
+++ b/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/TestADLResponseData.java
@@ -0,0 +1,120 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+
+package org.apache.hadoop.fs.adl;
+
+import org.apache.hadoop.fs.FileStatus;
+
+import java.util.Random;
+
+/**
+ * This class is responsible to provide generic test methods for mock up test
+ * to generate stub response for a network request.
+ */
+public final class TestADLResponseData {
+
+  private TestADLResponseData() {}
+
+  public static String getGetFileStatusJSONResponse(FileStatus status) {
+    String str = "{\"FileStatus\":{\"length\":" + status.getLen() + "," +
+        "\"pathSuffix\":\"\",\"type\":\"" + (status.isDirectory() ?
+        "DIRECTORY" :
+        "FILE") + "\"" +
+        ",\"blockSize\":" + status.getBlockSize() + ",\"accessTime\":" +
+        status.getAccessTime() + ",\"modificationTime\":" + status
+        .getModificationTime() + "" +
+        ",\"replication\":" + status.getReplication() + ",\"permission\":\""
+        + status.getPermission() + "\",\"owner\":\"" + status.getOwner()
+        + "\",\"group\":\"" + status.getGroup() + "\"}}";
+
+    return str;
+  }
+
+  public static String getGetFileStatusJSONResponse() {
+    return getGetFileStatusJSONResponse(4194304);
+  }
+
+  public static String getGetFileStatusJSONResponse(long length) {
+    String str = "{\"FileStatus\":{\"length\":" + length + "," +
+        "\"pathSuffix\":\"\",\"type\":\"FILE\",\"blockSize\":268435456," +
+        "\"accessTime\":1452103827023,\"modificationTime\":1452103827023," +
+        "\"replication\":0,\"permission\":\"777\"," +
+        "\"owner\":\"NotSupportYet\",\"group\":\"NotSupportYet\"}}";
+    return str;
+  }
+
+  public static String getListFileStatusJSONResponse(int dirSize) {
+    String list = "";
+    for (int i = 0; i < dirSize; ++i) {
+      list += "{\"length\":1024,\"pathSuffix\":\"" + java.util.UUID.randomUUID()
+          + "\",\"type\":\"FILE\",\"blockSize\":268435456," +
+          "\"accessTime\":1452103878833," +
+          "\"modificationTime\":1452103879190,\"replication\":0," +
+          "\"permission\":\"777\",\"owner\":\"NotSupportYet\"," +
+          "\"group\":\"NotSupportYet\"},";
+    }
+
+    list = list.substring(0, list.length() - 1);
+    String str = "{\"FileStatuses\":{\"FileStatus\":[" + list + "]}}";
+
+    return str;
+  }
+
+  public static String getJSONResponse(boolean status) {
+    String str = "{\"boolean\":" + status + "}";
+    return str;
+  }
+
+  public static String getErrorIllegalArgumentExceptionJSONResponse() {
+    String str = "{\n" +
+        "  \"RemoteException\":\n" +
+        "  {\n" +
+        "    \"exception\"    : \"IllegalArgumentException\",\n" +
+        "    \"javaClassName\": \"java.lang.IllegalArgumentException\",\n" +
+        "    \"message\"      : \"Bad Offset 0x83090015\"" +
+        "  }\n" +
+        "}";
+
+    return str;
+  }
+
+  public static String getErrorInternalServerExceptionJSONResponse() {
+    String str = "{\n" +
+        "  \"RemoteException\":\n" +
+        "  {\n" +
+        "    \"exception\"    : \"RumtimeException\",\n" +
+        "    \"javaClassName\": \"java.lang.RumtimeException\",\n" +
+        "    \"message\"      : \"Internal Server Error\"" +
+        "  }\n" +
+        "}";
+
+    return str;
+  }
+
+  public static byte[] getRandomByteArrayData() {
+    return getRandomByteArrayData(4 * 1024 * 1024);
+  }
+
+  public static byte[] getRandomByteArrayData(int size) {
+    byte[] b = new byte[size];
+    Random rand = new Random();
+    rand.nextBytes(b);
+    return b;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c9e71382/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/TestGetFileStatus.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/TestGetFileStatus.java b/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/TestGetFileStatus.java
new file mode 100644
index 0000000..4268b27
--- /dev/null
+++ b/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/TestGetFileStatus.java
@@ -0,0 +1,65 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+
+package org.apache.hadoop.fs.adl;
+
+import com.squareup.okhttp.mockwebserver.MockResponse;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.common.AdlMockWebServer;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.util.Time;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.net.URISyntaxException;
+
+/**
+ * This class is responsible for testing local getFileStatus implementation
+ * to cover correct parsing of successful and error JSON response
+ * from the server.
+ * Adls GetFileStatus operation is in detail covered in
+ * org.apache.hadoop.fs.adl.live testing package.
+ */
+public class TestGetFileStatus extends AdlMockWebServer {
+
+  @Test
+  public void getFileStatusReturnsAsExpected()
+      throws URISyntaxException, IOException {
+    getMockServer().enqueue(new MockResponse().setResponseCode(200)
+        .setBody(TestADLResponseData.getGetFileStatusJSONResponse()));
+    long startTime = Time.monotonicNow();
+    FileStatus fileStatus = getMockAdlFileSystem().getFileStatus(
+        new Path("/test1/test2"));
+    long endTime = Time.monotonicNow();
+    System.out.println("Time : " + (endTime - startTime));
+    Assert.assertTrue(fileStatus.isFile());
+    Assert.assertEquals(fileStatus.getPath().toString(),
+        "adl://" + getMockServer().getHostName() + ":"
+            + getMockServer().getPort()
+            + "/test1/test2");
+    Assert.assertEquals(fileStatus.getLen(), 4194304);
+    Assert.assertEquals(fileStatus.getBlockSize(), 268435456);
+    Assert.assertEquals(fileStatus.getReplication(), 0);
+    Assert.assertEquals(fileStatus.getPermission(), new FsPermission("777"));
+    Assert.assertEquals(fileStatus.getOwner(), "NotSupportYet");
+    Assert.assertEquals(fileStatus.getGroup(), "NotSupportYet");
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c9e71382/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/TestListStatus.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/TestListStatus.java b/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/TestListStatus.java
new file mode 100644
index 0000000..82c2494
--- /dev/null
+++ b/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/TestListStatus.java
@@ -0,0 +1,95 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+
+package org.apache.hadoop.fs.adl;
+
+import com.squareup.okhttp.mockwebserver.MockResponse;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.common.AdlMockWebServer;
+import org.apache.hadoop.util.Time;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.io.IOException;
+
+/**
+ * This class is responsible for testing local listStatus implementation to
+ * cover correct parsing of successful and error JSON response from the server.
+ * Adls ListStatus functionality is in detail covered in
+ * org.apache.hadoop.fs.adl.live testing package.
+ */
+public class TestListStatus extends AdlMockWebServer {
+
+  @Test
+  public void listStatusReturnsAsExpected() throws IOException {
+    getMockServer().enqueue(new MockResponse().setResponseCode(200)
+        .setBody(TestADLResponseData.getListFileStatusJSONResponse(10)));
+    long startTime = Time.monotonicNow();
+    FileStatus[] ls = getMockAdlFileSystem().listStatus(
+        new Path("/test1/test2"));
+    long endTime = Time.monotonicNow();
+    System.out.println("Time : " + (endTime - startTime));
+    Assert.assertEquals(ls.length, 10);
+
+    getMockServer().enqueue(new MockResponse().setResponseCode(200)
+        .setBody(TestADLResponseData.getListFileStatusJSONResponse(200)));
+    startTime = Time.monotonicNow();
+    ls = getMockAdlFileSystem().listStatus(new Path("/test1/test2"));
+    endTime = Time.monotonicNow();
+    System.out.println("Time : " + (endTime - startTime));
+    Assert.assertEquals(ls.length, 200);
+
+    getMockServer().enqueue(new MockResponse().setResponseCode(200)
+        .setBody(TestADLResponseData.getListFileStatusJSONResponse(2048)));
+    startTime = Time.monotonicNow();
+    ls = getMockAdlFileSystem().listStatus(new Path("/test1/test2"));
+    endTime = Time.monotonicNow();
+    System.out.println("Time : " + (endTime - startTime));
+    Assert.assertEquals(ls.length, 2048);
+  }
+
+  @Test
+  public void listStatusonFailure() throws IOException {
+    getMockServer().enqueue(new MockResponse().setResponseCode(403).setBody(
+        TestADLResponseData.getErrorIllegalArgumentExceptionJSONResponse()));
+    FileStatus[] ls = null;
+    long startTime = Time.monotonicNow();
+    try {
+      ls = getMockAdlFileSystem().listStatus(new Path("/test1/test2"));
+    } catch (IOException e) {
+      Assert.assertTrue(e.getMessage().contains("Bad Offset 0x83090015"));
+    }
+    long endTime = Time.monotonicNow();
+    System.out.println("Time : " + (endTime - startTime));
+
+    getMockServer().enqueue(new MockResponse().setResponseCode(500)
+        .setBody(
+            TestADLResponseData.getErrorInternalServerExceptionJSONResponse()));
+    startTime = Time.monotonicNow();
+    try {
+      ls = getMockAdlFileSystem().listStatus(new Path("/test1/test2"));
+    } catch (IOException e) {
+      Assert.assertTrue(e.getMessage().contains("Internal Server Error"));
+    }
+    endTime = Time.monotonicNow();
+    System.out.println("Time : " + (endTime - startTime));
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c9e71382/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/TestableAdlFileSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/TestableAdlFileSystem.java b/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/TestableAdlFileSystem.java
new file mode 100644
index 0000000..1e03bbf
--- /dev/null
+++ b/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/TestableAdlFileSystem.java
@@ -0,0 +1,31 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+
+package org.apache.hadoop.fs.adl;
+
+/**
+ * This class overrides AdlFileSystem to change transport scheme to http instead
+ * of https to run against Mock Server.
+ */
+public class TestableAdlFileSystem extends AdlFileSystem {
+  @Override
+  protected String getTransportScheme() {
+    return "http";
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c9e71382/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/AdlStorageConfiguration.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/AdlStorageConfiguration.java b/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/AdlStorageConfiguration.java
new file mode 100644
index 0000000..d257768
--- /dev/null
+++ b/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/AdlStorageConfiguration.java
@@ -0,0 +1,76 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+
+package org.apache.hadoop.fs.adl.live;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.adl.AdlFileSystem;
+
+import java.io.IOException;
+import java.net.URI;
+import java.net.URISyntaxException;
+
+/**
+ * Utility class to configure real Adls storage to run Live test suite against.
+ */
+public final class AdlStorageConfiguration {
+  private AdlStorageConfiguration() {}
+
+  private static final String CONTRACT_ENABLE_KEY =
+      "dfs.adl.test.contract" + ".enable";
+  private static final String TEST_CONFIGURATION_FILE_NAME =
+      "contract-test-options.xml";
+  private static final String TEST_SUPPORTED_TEST_CONFIGURATION_FILE_NAME =
+      "adls.xml";
+
+  private static boolean isContractTestEnabled = false;
+  private static Configuration conf = null;
+
+  public static Configuration getConfiguration() {
+    Configuration localConf = new Configuration();
+    localConf.addResource(TEST_CONFIGURATION_FILE_NAME);
+    localConf.addResource(TEST_SUPPORTED_TEST_CONFIGURATION_FILE_NAME);
+    return localConf;
+  }
+
+  public static boolean isContractTestEnabled() {
+    if (conf == null) {
+      conf = getConfiguration();
+    }
+
+    isContractTestEnabled = conf.getBoolean(CONTRACT_ENABLE_KEY, false);
+    return isContractTestEnabled;
+  }
+
+  public static FileSystem createAdlStorageConnector()
+      throws URISyntaxException, IOException {
+    if (conf == null) {
+      conf = getConfiguration();
+    }
+
+    if(!isContractTestEnabled()) {
+      return null;
+    }
+
+    AdlFileSystem fileSystem = new AdlFileSystem();
+    fileSystem.initialize(new URI(conf.get("fs.defaultFS")), conf);
+    return fileSystem;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c9e71382/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/AdlStorageContract.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/AdlStorageContract.java b/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/AdlStorageContract.java
new file mode 100644
index 0000000..1e40199
--- /dev/null
+++ b/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/AdlStorageContract.java
@@ -0,0 +1,65 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+
+package org.apache.hadoop.fs.adl.live;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.contract.AbstractFSContract;
+
+import java.io.IOException;
+import java.net.URISyntaxException;
+
+/**
+ * Extension of AbstractFSContract representing a filesystem contract that
+ * a Adls filesystem implementation is expected implement.
+ */
+public class AdlStorageContract extends AbstractFSContract {
+  private FileSystem fs;
+
+  protected AdlStorageContract(Configuration conf) {
+    super(conf);
+    try {
+      fs = AdlStorageConfiguration.createAdlStorageConnector();
+    } catch (URISyntaxException e) {
+      throw new IllegalStateException("Can not initialize ADL FileSystem. "
+          + "Please check fs.defaultFS property.", e);
+    } catch (IOException e) {
+      throw new IllegalStateException("Can not initialize ADL FileSystem.", e);
+    }
+    this.setConf(AdlStorageConfiguration.getConfiguration());
+  }
+
+  @Override
+  public String getScheme() {
+    return "adl";
+  }
+
+  @Override
+  public FileSystem getTestFileSystem() throws IOException {
+    return this.fs;
+  }
+
+  @Override
+  public Path getTestPath() {
+    Path path = new Path("/test");
+    return path;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c9e71382/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestAdlContractAppendLive.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestAdlContractAppendLive.java b/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestAdlContractAppendLive.java
new file mode 100644
index 0000000..83390af
--- /dev/null
+++ b/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestAdlContractAppendLive.java
@@ -0,0 +1,53 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+
+package org.apache.hadoop.fs.adl.live;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.contract.AbstractContractAppendTest;
+import org.apache.hadoop.fs.contract.AbstractFSContract;
+import org.apache.hadoop.fs.contract.ContractTestUtils;
+import org.junit.Before;
+import org.junit.Test;
+
+/**
+ * Verify Adls APPEND semantics compliance with Hadoop.
+ */
+public class TestAdlContractAppendLive extends AbstractContractAppendTest {
+
+  @Override
+  protected AbstractFSContract createContract(Configuration configuration) {
+    return new AdlStorageContract(configuration);
+  }
+
+  @Override
+  @Test
+  public void testRenameFileBeingAppended() throws Throwable {
+    ContractTestUtils.unsupported("Skipping since renaming file in append "
+        + "mode not supported in Adl");
+  }
+
+  @Before
+  @Override
+  public void setup() throws Exception {
+    org.junit.Assume
+        .assumeTrue(AdlStorageConfiguration.isContractTestEnabled());
+    super.setup();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c9e71382/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestAdlContractConcatLive.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestAdlContractConcatLive.java b/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestAdlContractConcatLive.java
new file mode 100644
index 0000000..8474e9c
--- /dev/null
+++ b/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestAdlContractConcatLive.java
@@ -0,0 +1,52 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+
+package org.apache.hadoop.fs.adl.live;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.contract.AbstractContractConcatTest;
+import org.apache.hadoop.fs.contract.AbstractFSContract;
+import org.apache.hadoop.fs.contract.ContractTestUtils;
+import org.junit.Before;
+import org.junit.Test;
+
+/**
+ * Verify Adls CONCAT semantics compliance with Hadoop.
+ */
+public class TestAdlContractConcatLive extends AbstractContractConcatTest {
+
+  @Override
+  protected AbstractFSContract createContract(Configuration configuration) {
+    return new AdlStorageContract(configuration);
+  }
+
+  @Before
+  @Override
+  public void setup() throws Exception {
+    org.junit.Assume
+        .assumeTrue(AdlStorageConfiguration.isContractTestEnabled());
+    super.setup();
+  }
+
+  @Test
+  public void testConcatMissingTarget() throws Throwable {
+    ContractTestUtils.unsupported("BUG : Adl to support expectation from "
+        + "concat on missing targets.");
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c9e71382/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestAdlContractCreateLive.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestAdlContractCreateLive.java b/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestAdlContractCreateLive.java
new file mode 100644
index 0000000..907c50c
--- /dev/null
+++ b/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestAdlContractCreateLive.java
@@ -0,0 +1,52 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+
+package org.apache.hadoop.fs.adl.live;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.contract.AbstractContractCreateTest;
+import org.apache.hadoop.fs.contract.AbstractFSContract;
+import org.apache.hadoop.fs.contract.ContractTestUtils;
+import org.junit.Before;
+import org.junit.Test;
+
+/**
+ * Verify Adls CREATE semantics compliance with Hadoop.
+ */
+public class TestAdlContractCreateLive extends AbstractContractCreateTest {
+
+  @Override
+  protected AbstractFSContract createContract(Configuration configuration) {
+    return new AdlStorageContract(configuration);
+  }
+
+  @Before
+  @Override
+  public void setup() throws Exception {
+    org.junit.Assume
+        .assumeTrue(AdlStorageConfiguration.isContractTestEnabled());
+    super.setup();
+  }
+
+  @Test
+  public void testOverwriteEmptyDirectory() throws Throwable {
+    ContractTestUtils
+        .unsupported("BUG : Adl to support override empty " + "directory.");
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c9e71382/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestAdlContractDeleteLive.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestAdlContractDeleteLive.java b/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestAdlContractDeleteLive.java
new file mode 100644
index 0000000..30eaec7
--- /dev/null
+++ b/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestAdlContractDeleteLive.java
@@ -0,0 +1,44 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+
+package org.apache.hadoop.fs.adl.live;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.contract.AbstractContractDeleteTest;
+import org.apache.hadoop.fs.contract.AbstractFSContract;
+import org.junit.Before;
+
+/**
+ * Verify Adls DELETE semantics compliance with Hadoop.
+ */
+public class TestAdlContractDeleteLive extends AbstractContractDeleteTest {
+
+  @Override
+  protected AbstractFSContract createContract(Configuration configuration) {
+    return new AdlStorageContract(configuration);
+  }
+
+  @Before
+  @Override
+  public void setup() throws Exception {
+    org.junit.Assume
+        .assumeTrue(AdlStorageConfiguration.isContractTestEnabled());
+    super.setup();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c9e71382/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestAdlContractMkdirLive.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestAdlContractMkdirLive.java b/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestAdlContractMkdirLive.java
new file mode 100644
index 0000000..e498110
--- /dev/null
+++ b/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestAdlContractMkdirLive.java
@@ -0,0 +1,55 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+
+package org.apache.hadoop.fs.adl.live;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.contract.AbstractContractMkdirTest;
+import org.apache.hadoop.fs.contract.AbstractFSContract;
+import org.apache.hadoop.fs.contract.ContractTestUtils;
+import org.junit.Before;
+import org.junit.Test;
+
+/**
+ * Verify Adls MKDIR semantics compliance with Hadoop.
+ */
+public class TestAdlContractMkdirLive extends AbstractContractMkdirTest {
+  @Override
+  protected AbstractFSContract createContract(Configuration conf) {
+    return new AdlStorageContract(conf);
+  }
+
+  @Before
+  @Override
+  public void setup() throws Exception {
+    org.junit.Assume
+        .assumeTrue(AdlStorageConfiguration.isContractTestEnabled());
+    super.setup();
+  }
+
+  @Test
+  public void testMkdirOverParentFile() throws Throwable {
+    ContractTestUtils.unsupported("Not supported by Adl");
+  }
+
+  @Test
+  public void testNoMkdirOverFile() throws Throwable {
+    ContractTestUtils.unsupported("Not supported by Adl");
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c9e71382/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestAdlContractOpenLive.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestAdlContractOpenLive.java b/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestAdlContractOpenLive.java
new file mode 100644
index 0000000..2bb2095
--- /dev/null
+++ b/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestAdlContractOpenLive.java
@@ -0,0 +1,44 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+
+package org.apache.hadoop.fs.adl.live;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.contract.AbstractContractOpenTest;
+import org.apache.hadoop.fs.contract.AbstractFSContract;
+import org.junit.Before;
+
+/**
+ * Verify Adls OPEN/READ semantics compliance with Hadoop.
+ */
+public class TestAdlContractOpenLive extends AbstractContractOpenTest {
+
+  @Override
+  protected AbstractFSContract createContract(Configuration configuration) {
+    return new AdlStorageContract(configuration);
+  }
+
+  @Before
+  @Override
+  public void setup() throws Exception {
+    org.junit.Assume
+        .assumeTrue(AdlStorageConfiguration.isContractTestEnabled());
+    super.setup();
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c9e71382/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestAdlContractRenameLive.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestAdlContractRenameLive.java b/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestAdlContractRenameLive.java
new file mode 100644
index 0000000..06063c5
--- /dev/null
+++ b/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestAdlContractRenameLive.java
@@ -0,0 +1,63 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+
+package org.apache.hadoop.fs.adl.live;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.contract.AbstractContractRenameTest;
+import org.apache.hadoop.fs.contract.AbstractFSContract;
+import org.apache.hadoop.fs.contract.ContractTestUtils;
+import org.junit.Before;
+import org.junit.Test;
+
+/**
+ * Verify Adls RENAME semantics compliance with Hadoop.
+ */
+public class TestAdlContractRenameLive extends AbstractContractRenameTest {
+
+  @Override
+  protected AbstractFSContract createContract(Configuration configuration) {
+    return new AdlStorageContract(configuration);
+  }
+
+  @Before
+  @Override
+  public void setup() throws Exception {
+    org.junit.Assume
+        .assumeTrue(AdlStorageConfiguration.isContractTestEnabled());
+    super.setup();
+  }
+
+  @Test
+  public void testRenameFileOverExistingFile() throws Throwable {
+    ContractTestUtils
+        .unsupported("BUG : Adl to support full complete POSIX" + "behaviour");
+  }
+
+  @Test
+  public void testRenameFileNonexistentDir() throws Throwable {
+    ContractTestUtils
+        .unsupported("BUG : Adl to support create dir is not " + "exist");
+  }
+
+  @Test
+  public void testRenameWithNonEmptySubDir() throws Throwable {
+    ContractTestUtils.unsupported("BUG : Adl to support non empty dir move.");
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c9e71382/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestAdlContractRootDirLive.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestAdlContractRootDirLive.java b/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestAdlContractRootDirLive.java
new file mode 100644
index 0000000..bf4e549
--- /dev/null
+++ b/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestAdlContractRootDirLive.java
@@ -0,0 +1,52 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+
+package org.apache.hadoop.fs.adl.live;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.contract.AbstractContractRootDirectoryTest;
+import org.apache.hadoop.fs.contract.AbstractFSContract;
+import org.apache.hadoop.fs.contract.ContractTestUtils;
+import org.junit.Before;
+import org.junit.Test;
+
+/**
+ * Verify Adls root level operation support.
+ */
+public class TestAdlContractRootDirLive
+    extends AbstractContractRootDirectoryTest {
+  @Override
+  protected AbstractFSContract createContract(Configuration configuration) {
+    return new AdlStorageContract(configuration);
+  }
+
+  @Before
+  @Override
+  public void setup() throws Exception {
+    org.junit.Assume
+        .assumeTrue(AdlStorageConfiguration.isContractTestEnabled());
+    super.setup();
+  }
+
+  @Test
+  public void testRmNonEmptyRootDirNonRecursive() throws Throwable {
+    ContractTestUtils.unsupported(
+        "BUG : Adl should throw exception instred " + "of returning false.");
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c9e71382/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestAdlContractSeekLive.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestAdlContractSeekLive.java b/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestAdlContractSeekLive.java
new file mode 100644
index 0000000..0976464
--- /dev/null
+++ b/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestAdlContractSeekLive.java
@@ -0,0 +1,44 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+
+package org.apache.hadoop.fs.adl.live;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.contract.AbstractContractSeekTest;
+import org.apache.hadoop.fs.contract.AbstractFSContract;
+import org.junit.Before;
+
+/**
+ * Verify Adls OPEN/READ seek operation support.
+ */
+public class TestAdlContractSeekLive extends AbstractContractSeekTest {
+
+  @Override
+  protected AbstractFSContract createContract(Configuration configuration) {
+    return new AdlStorageContract(configuration);
+  }
+
+  @Before
+  @Override
+  public void setup() throws Exception {
+    org.junit.Assume
+        .assumeTrue(AdlStorageConfiguration.isContractTestEnabled());
+    super.setup();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c9e71382/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestAdlDifferentSizeWritesLive.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestAdlDifferentSizeWritesLive.java b/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestAdlDifferentSizeWritesLive.java
new file mode 100644
index 0000000..b712995
--- /dev/null
+++ b/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestAdlDifferentSizeWritesLive.java
@@ -0,0 +1,102 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+
+package org.apache.hadoop.fs.adl.live;
+
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.net.URISyntaxException;
+import java.util.Random;
+
+/**
+ * Verify different data segment size writes ensure the integrity and
+ * order of the data.
+ */
+public class TestAdlDifferentSizeWritesLive {
+
+  public static byte[] getRandomByteArrayData(int size) {
+    byte[] b = new byte[size];
+    Random rand = new Random();
+    rand.nextBytes(b);
+    return b;
+  }
+
+  @Before
+  public void setup() throws Exception {
+    org.junit.Assume
+        .assumeTrue(AdlStorageConfiguration.isContractTestEnabled());
+  }
+
+  @Test
+  public void testSmallDataWrites() throws IOException {
+    testDataIntegrity(4 * 1024 * 1024, 1 * 1024);
+    testDataIntegrity(4 * 1024 * 1024, 7 * 1024);
+    testDataIntegrity(4 * 1024 * 1024, 10);
+    testDataIntegrity(2 * 1024 * 1024, 10);
+    testDataIntegrity(1 * 1024 * 1024, 10);
+    testDataIntegrity(100, 1);
+  }
+
+  @Test
+  public void testMediumDataWrites() throws IOException {
+    testDataIntegrity(4 * 1024 * 1024, 1 * 1024 * 1024);
+    testDataIntegrity(7 * 1024 * 1024, 2 * 1024 * 1024);
+    testDataIntegrity(9 * 1024 * 1024, 2 * 1024 * 1024);
+    testDataIntegrity(10 * 1024 * 1024, 3 * 1024 * 1024);
+  }
+
+  private void testDataIntegrity(int totalSize, int chunkSize)
+      throws IOException {
+    Path path = new Path("/test/dataIntegrityCheck");
+    FileSystem fs = null;
+    try {
+      fs = AdlStorageConfiguration.createAdlStorageConnector();
+    } catch (URISyntaxException e) {
+      throw new IllegalStateException("Can not initialize ADL FileSystem. "
+          + "Please check fs.defaultFS property.", e);
+    }
+    byte[] expectedData = getRandomByteArrayData(totalSize);
+
+    FSDataOutputStream out = fs.create(path, true);
+    int iteration = totalSize / chunkSize;
+    int reminderIteration = totalSize % chunkSize;
+    int offset = 0;
+    for (int i = 0; i < iteration; ++i) {
+      out.write(expectedData, offset, chunkSize);
+      offset += chunkSize;
+    }
+
+    out.write(expectedData, offset, reminderIteration);
+    out.close();
+
+    byte[] actualData = new byte[totalSize];
+    FSDataInputStream in = fs.open(path);
+    in.readFully(0, actualData);
+    in.close();
+    Assert.assertArrayEquals(expectedData, actualData);
+    Assert.assertTrue(fs.delete(path, true));
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c9e71382/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestAdlFileSystemContractLive.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestAdlFileSystemContractLive.java b/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestAdlFileSystemContractLive.java
new file mode 100644
index 0000000..e007c4e
--- /dev/null
+++ b/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestAdlFileSystemContractLive.java
@@ -0,0 +1,111 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+
+package org.apache.hadoop.fs.adl.live;
+
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.FileSystemContractBaseTest;
+import org.apache.hadoop.fs.Path;
+import org.junit.Test;
+
+import java.io.IOException;
+
+/**
+ * Verify Adls adhere to Hadoop file system semantics.
+ */
+public class TestAdlFileSystemContractLive extends FileSystemContractBaseTest {
+  private FileSystem adlStore;
+
+  @Override
+  protected void setUp() throws Exception {
+    adlStore = AdlStorageConfiguration.createAdlStorageConnector();
+    if (AdlStorageConfiguration.isContractTestEnabled()) {
+      fs = adlStore;
+    }
+  }
+
+  @Override
+  protected void tearDown() throws Exception {
+    if (AdlStorageConfiguration.isContractTestEnabled()) {
+      cleanup();
+      adlStore = null;
+      fs = null;
+    }
+  }
+
+  private void cleanup() throws IOException {
+    adlStore.delete(new Path("/test"), true);
+  }
+
+  @Override
+  protected void runTest() throws Throwable {
+    if (AdlStorageConfiguration.isContractTestEnabled()) {
+      super.runTest();
+    }
+  }
+
+  public void testGetFileStatus() throws IOException {
+    if (!AdlStorageConfiguration.isContractTestEnabled()) {
+      return;
+    }
+
+    Path testPath = new Path("/test/adltest");
+    if (adlStore.exists(testPath)) {
+      adlStore.delete(testPath, false);
+    }
+
+    adlStore.create(testPath).close();
+    assertTrue(adlStore.delete(testPath, false));
+  }
+
+  /**
+   * The following tests are failing on Azure Data Lake and the Azure Data Lake
+   * file system code needs to be modified to make them pass.
+   * A separate work item has been opened for this.
+   */
+  @Test
+  @Override
+  public void testMkdirsFailsForSubdirectoryOfExistingFile() throws Exception {
+    // BUG : Adl should return exception instead of false.
+  }
+
+  @Test
+  @Override
+  public void testMkdirsWithUmask() throws Exception {
+    // Support under implementation in Adl
+  }
+
+  @Test
+  @Override
+  public void testMoveFileUnderParent() throws Exception {
+    // BUG: Adl server should return expected status code.
+  }
+
+  @Test
+  @Override
+  public void testRenameFileToSelf() throws Exception {
+    // BUG: Adl server should return expected status code.
+  }
+
+  @Test
+  @Override
+  public void testRenameToDirWithSamePrefixAllowed() throws Exception {
+    // BUG: Adl server should return expected status code.
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c9e71382/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestAdlReadLive.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestAdlReadLive.java b/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestAdlReadLive.java
new file mode 100644
index 0000000..f1d909c
--- /dev/null
+++ b/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestAdlReadLive.java
@@ -0,0 +1,342 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+
+package org.apache.hadoop.fs.adl.live;
+
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.net.URISyntaxException;
+import java.util.Random;
+import java.util.UUID;
+
+/**
+ * Verify different data segment size read from the file to ensure the
+ * integrity and order of the data over
+ * BufferManger and BatchByteArrayInputStream implementation.
+ */
+public class TestAdlReadLive {
+  private String expectedData = "1234567890abcdefghijklmnopqrstuvwxyz";
+
+  @Before
+  public void setup() throws Exception {
+    org.junit.Assume
+        .assumeTrue(AdlStorageConfiguration.isContractTestEnabled());
+  }
+
+  private FileSystem getFileSystem() throws IOException, URISyntaxException {
+    return AdlStorageConfiguration.createAdlStorageConnector();
+  }
+
+  private void setupFile(Path path) throws IOException, URISyntaxException {
+    setupFile(path, expectedData);
+  }
+
+  private void setupFile(Path path, String data)
+      throws IOException, URISyntaxException {
+    expectedData = data;
+    FileSystem fs = getFileSystem();
+    fs.delete(path, true);
+    FSDataOutputStream fdis = fs.create(path);
+    fdis.writeBytes(expectedData);
+    fdis.close();
+    fs.listStatus(path.getParent());
+    long actualLen = fs.getFileStatus(path).getLen();
+    long expectedLen = expectedData.length();
+    System.out.println(
+        " Length of file : " + fs.getFileStatus(path).getLen() + " " + fs
+            .getUri());
+    Assert.assertEquals(expectedLen, actualLen);
+  }
+
+  @Test
+  public void
+      testOpenReadMoreThanAvailableBufferCrashFixIndexOutOfBoundsException()
+      throws Throwable {
+    Path path = new Path("/test1");
+    FileSystem fs = getFileSystem();
+    setupFile(path);
+
+    if (fs.exists(path)) {
+      Assert.assertTrue(fs.delete(path, true));
+    }
+
+    FSDataOutputStream outputStream = fs.create(path);
+    final byte[] data = new byte[24 * 1024 * 1024];
+    Random ran = new Random();
+    ran.nextBytes(data);
+    outputStream.write(data);
+
+    FSDataInputStream bb = fs.open(path);
+    byte[] expected = new byte[4 * 1024 * 1024];
+    bb.read();
+    bb.readFully(16711581, expected, 33,
+        65640); // BugFix : Was causing crash IndexOutOfBoundsException
+    bb.seek(16711581);
+    bb.readFully(16711576, expected, 33, 65640);
+    bb.readFully(16711578, expected, 33, 65640);
+    bb.readFully(16711580, expected, 33, 65640);
+    bb.readFully(16711576, expected, 0, expected.length);
+    bb.seek(0);
+    expected = new byte[134144];
+    while (bb.read() != -1){
+      continue;
+    }
+    bb.readFully(0, data, 0, data.length);
+  }
+
+  @Test
+  public void readNullData() throws IOException, URISyntaxException {
+    String data = "SPL   \u0001Lorg.apache.hadoop.examples.terasort"
+        + ".TeraGen$RangeInputFormat$RangeInputSplit \u008DLK@Lorg.apache"
+        + ".hadoop.examples.terasort"
+        + ".TeraGen$RangeInputFormat$RangeInputSplit\u008DLK@\u008DLK@";
+    Path path = new Path("/test4");
+    FileSystem fs = this.getFileSystem();
+    setupFile(path, data);
+    FSDataInputStream bb = fs.open(path);
+    int i = 0;
+    String actualData = new String();
+    System.out.println("Data Length :" + expectedData.length());
+    byte[] arr = new byte[data.length()];
+    bb.readFully(0, arr);
+    actualData = new String(arr);
+    System.out.println(" Data : " + actualData);
+    Assert.assertEquals(actualData.length(), expectedData.length());
+
+    arr = new byte[data.length() - 7];
+    bb.readFully(7, arr);
+    actualData = new String(arr);
+    Assert.assertEquals(actualData.length(), expectedData.length() - 7);
+    bb.close();
+  }
+
+  @Test
+  public void readTest() throws IOException, URISyntaxException {
+    Path path = new Path("/test4");
+    FileSystem fs = this.getFileSystem();
+    setupFile(path);
+    FSDataInputStream bb = fs.open(path);
+    int i = 0;
+    String actualData = new String();
+    while (true) {
+      int c = bb.read();
+      if (c < 0) {
+        break;
+      }
+      actualData += (char) c;
+    }
+
+    byte[] b = new byte[100];
+    System.out.println(bb.read(b, 9, 91));
+    System.out.println(bb.read());
+    System.out.println(bb.read());
+    System.out.println(bb.read());
+    System.out.println(bb.read());
+    System.out.println(bb.read());
+    System.out.println(bb.read());
+
+    bb.close();
+    Assert.assertEquals(actualData, expectedData);
+
+    for (int j = 0; j < 100; ++j) {
+      fs = this.getFileSystem();
+      fs.exists(new Path("/test" + j));
+    }
+  }
+
+  @Test
+  public void readByteTest() throws IOException, URISyntaxException {
+    Path path = new Path("/test3");
+    FileSystem fs = this.getFileSystem();
+    setupFile(path);
+    FSDataInputStream bb = fs.open(path);
+    int i = 0;
+    byte[] data = new byte[expectedData.length()];
+    int readByte = bb.read(data);
+    bb.close();
+    Assert.assertEquals(readByte, expectedData.length());
+    Assert.assertEquals(new String(data), expectedData);
+  }
+
+  @Test
+  public void readByteFullyTest() throws IOException, URISyntaxException {
+    Path path = new Path("/test2");
+    FileSystem fs = this.getFileSystem();
+    setupFile(path);
+    FSDataInputStream bb = fs.open(path);
+    int i = 0;
+    byte[] data = new byte[expectedData.length()];
+    bb.readFully(data);
+    bb.close();
+    Assert.assertEquals(new String(data), expectedData);
+
+    bb = fs.open(path);
+    bb.readFully(data, 0, data.length);
+    bb.close();
+    Assert.assertEquals(new String(data), expectedData);
+  }
+
+  @Test
+  public void readCombinationTest() throws IOException, URISyntaxException {
+    Path path = new Path("/test1");
+    FileSystem fs = this.getFileSystem();
+    setupFile(path);
+    FSDataInputStream bb = fs.open(path);
+    int i = 0;
+    byte[] data = new byte[5];
+    int readByte = bb.read(data);
+    Assert.assertEquals(new String(data), expectedData.substring(0, 5));
+
+    bb.readFully(data, 0, data.length);
+    Assert.assertEquals(new String(data), expectedData.substring(5, 10));
+    bb.close();
+    bb = fs.open(path);
+    bb.readFully(5, data, 0, data.length);
+    Assert.assertEquals(new String(data), expectedData.substring(5, 10));
+
+    bb.read(data);
+    Assert.assertEquals(new String(data), expectedData.substring(0, 5));
+    bb.close();
+    bb = fs.open(path);
+    bb.read(new byte[100]);
+    bb.close();
+  }
+
+  @Test
+  public void readMultiSeekTest() throws IOException, URISyntaxException {
+    final Path path = new Path(
+        "/delete14/" + UUID.randomUUID().toString().replaceAll("-", ""));
+    FileSystem fs = this.getFileSystem();
+
+    final byte[] actualData = new byte[3267397];
+    Random ran = new Random();
+    ran.nextBytes(actualData);
+    byte[] testData = null;
+
+    fs.delete(path, true);
+    FSDataOutputStream os = fs.create(path);
+    os.write(actualData);
+    os.close();
+
+    FSDataInputStream bb = fs.open(path);
+    byte[] data = new byte[16384];
+    bb.readFully(3251013, data, 0, 16384);
+    testData = new byte[16384];
+    System.arraycopy(actualData, 3251013, testData, 0, 16384);
+    Assert.assertArrayEquals(testData, data);
+
+    data = new byte[1921];
+    bb.readFully(3265476, data, 0, 1921);
+    testData = new byte[1921];
+    System.arraycopy(actualData, 3265476, testData, 0, 1921);
+    Assert.assertArrayEquals(testData, data);
+
+    data = new byte[3267394];
+    bb.readFully(3, data, 0, 3267394);
+    testData = new byte[3267394];
+    System.arraycopy(actualData, 3, testData, 0, 3267394);
+    Assert.assertArrayEquals(testData, data);
+
+    data = new byte[3266943];
+    bb.readFully(454, data, 0, 3266943);
+    testData = new byte[3266943];
+    System.arraycopy(actualData, 454, testData, 0, 3266943);
+    Assert.assertArrayEquals(testData, data);
+
+    data = new byte[3265320];
+    bb.readFully(2077, data, 0, 3265320);
+    testData = new byte[3265320];
+    System.arraycopy(actualData, 2077, testData, 0, 3265320);
+    Assert.assertArrayEquals(testData, data);
+
+    bb.close();
+
+    bb = fs.open(path);
+
+    data = new byte[3263262];
+    bb.readFully(4135, data, 0, 3263262);
+    testData = new byte[3263262];
+    System.arraycopy(actualData, 4135, testData, 0, 3263262);
+    Assert.assertArrayEquals(testData, data);
+
+    data = new byte[2992591];
+    bb.readFully(274806, data, 0, 2992591);
+    testData = new byte[2992591];
+    System.arraycopy(actualData, 274806, testData, 0, 2992591);
+    Assert.assertArrayEquals(testData, data);
+
+    data = new byte[1985665];
+    bb.readFully(1281732, data, 0, 1985665);
+    testData = new byte[1985665];
+    System.arraycopy(actualData, 1281732, testData, 0, 1985665);
+    Assert.assertArrayEquals(testData, data);
+
+    data = new byte[3267394];
+    try {
+      bb.readFully(2420207, data, 0, 3267394);
+      Assert.fail("EOF expected");
+    } catch (IOException e) {
+    }
+
+    bb.close();
+  }
+
+  @Test
+  public void allASCIICharTest() throws IOException, URISyntaxException {
+    final Path path = new Path(
+        "/delete14/" + UUID.randomUUID().toString().replaceAll("-", ""));
+    FileSystem fs = this.getFileSystem();
+    final byte[] actualData = new byte[127];
+    for (byte i = 0; i < 127; ++i) {
+      actualData[i] = i;
+    }
+
+    fs.delete(path, true);
+    FSDataOutputStream os = fs.create(path);
+    os.write(actualData);
+    os.close();
+
+    FSDataInputStream bb = fs.open(path);
+    byte[] data = new byte[127];
+
+    bb.readFully(0, data, 0, data.length);
+    bb.close();
+    Assert.assertArrayEquals(data, actualData);
+
+    bb = fs.open(path);
+    int byteRead = 1;
+    while (bb.read() != -1) {
+      byteRead++;
+    }
+
+    bb.seek(0);
+    byteRead = 1;
+    while (bb.read() != -1) {
+      byteRead++;
+    }
+    bb.close();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c9e71382/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestAdlWebHdfsFileContextCreateMkdirLive.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestAdlWebHdfsFileContextCreateMkdirLive.java b/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestAdlWebHdfsFileContextCreateMkdirLive.java
new file mode 100644
index 0000000..fc50c04
--- /dev/null
+++ b/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestAdlWebHdfsFileContextCreateMkdirLive.java
@@ -0,0 +1,79 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+
+package org.apache.hadoop.fs.adl.live;
+
+import org.apache.commons.lang.RandomStringUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.DelegateToFileSystem;
+import org.apache.hadoop.fs.FileContext;
+import org.apache.hadoop.fs.FileContextCreateMkdirBaseTest;
+import org.apache.hadoop.fs.FileContextTestHelper;
+import org.apache.hadoop.fs.FileSystem;
+import org.junit.Assume;
+import org.junit.BeforeClass;
+
+import java.io.File;
+import java.net.URI;
+
+/**
+ * Verify Adls file system adhere to Hadoop file system contract using bunch of
+ * available test in FileContextCreateMkdirBaseTest.
+ */
+public class TestAdlWebHdfsFileContextCreateMkdirLive
+    extends FileContextCreateMkdirBaseTest {
+  private static final String KEY_FILE_SYSTEM = "fs.defaultFS";
+
+  @Override
+  public void setUp() throws Exception {
+    Configuration conf = AdlStorageConfiguration.getConfiguration();
+    String fileSystem = conf.get(KEY_FILE_SYSTEM);
+    if (fileSystem == null || fileSystem.trim().length() == 0) {
+      throw new Exception("Default file system not configured.");
+    }
+    URI uri = new URI(fileSystem);
+    FileSystem fs = AdlStorageConfiguration.createAdlStorageConnector();
+    fc = FileContext.getFileContext(
+        new DelegateToFileSystem(uri, fs, conf, fs.getScheme(), false) {
+        }, conf);
+    super.setUp();
+  }
+
+  /**
+   * Required to override since the getRandmizedTestDir on Windows generates
+   * absolute path of the local file path which contains ":" character.
+   * Example file system path generated is "adl://<FileSystem Path>/d:/a/b/c
+   *
+   * Adls does not support : character in the path hence overriding to remove
+   * unsupported character from the path.
+   *
+   * @return FileContextTestHelper
+   */
+  @Override
+  protected FileContextTestHelper createFileContextHelper() {
+    return new FileContextTestHelper(new File(
+        RandomStringUtils.randomAlphanumeric(10))
+        .getAbsolutePath().replaceAll(":", ""));
+  }
+
+  @BeforeClass
+  public static void skipTestCheck() {
+    Assume.assumeTrue(AdlStorageConfiguration.isContractTestEnabled());
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c9e71382/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestAdlWebHdfsFileContextMainOperationsLive.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestAdlWebHdfsFileContextMainOperationsLive.java b/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestAdlWebHdfsFileContextMainOperationsLive.java
new file mode 100644
index 0000000..b135550
--- /dev/null
+++ b/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestAdlWebHdfsFileContextMainOperationsLive.java
@@ -0,0 +1,104 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+
+package org.apache.hadoop.fs.adl.live;
+
+import org.apache.commons.lang.RandomStringUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.DelegateToFileSystem;
+import org.apache.hadoop.fs.FileContext;
+import org.apache.hadoop.fs.FileContextMainOperationsBaseTest;
+import org.apache.hadoop.fs.FileContextTestHelper;
+import org.apache.hadoop.fs.FileSystem;
+import org.junit.Assume;
+import org.junit.BeforeClass;
+
+import java.io.File;
+import java.io.IOException;
+import java.net.URI;
+
+/**
+ * Verify Adls file system adhere to Hadoop file system contract using bunch of
+ * available test in FileContextMainOperationsBaseTest.
+ */
+public class TestAdlWebHdfsFileContextMainOperationsLive
+    extends FileContextMainOperationsBaseTest {
+
+  private static final String KEY_FILE_SYSTEM = "fs.defaultFS";
+
+  @BeforeClass
+  public static void skipTestCheck() {
+    Assume.assumeTrue(AdlStorageConfiguration.isContractTestEnabled());
+  }
+
+  @Override
+  public void setUp() throws Exception {
+    Configuration conf = AdlStorageConfiguration.getConfiguration();
+    String fileSystem = conf.get(KEY_FILE_SYSTEM);
+    if (fileSystem == null || fileSystem.trim().length() == 0) {
+      throw new Exception("Default file system not configured.");
+    }
+    URI uri = new URI(fileSystem);
+    FileSystem fs = AdlStorageConfiguration.createAdlStorageConnector();
+    fc = FileContext.getFileContext(
+        new DelegateToFileSystem(uri, fs, conf, fs.getScheme(), false) {
+        }, conf);
+    super.setUp();
+  }
+
+  /**
+   * Required to override since the getRandmizedTestDir on Windows generates
+   * absolute path of the local file path which contains ":" character.
+   * Example file system path generated is "adl://<FileSystem Path>/d:/a/b/c
+   *
+   * Adls does not support : character in the path hence overriding to remove
+   * unsupported character from the path.
+   *
+   * @return FileContextTestHelper
+   */
+  @Override
+  protected FileContextTestHelper createFileContextHelper() {
+    return new FileContextTestHelper(
+        new File(RandomStringUtils.randomAlphanumeric(10)).getAbsolutePath()
+            .replaceAll(":", ""));
+  }
+
+  @Override
+  protected boolean listCorruptedBlocksSupported() {
+    return false;
+  }
+
+  @Override
+  public void testUnsupportedSymlink() throws IOException {
+    Assume.assumeTrue("Symbolic link are not supported by Adls", false);
+  }
+
+  /**
+   * In case this test is causing failure due to
+   * java.lang.RuntimeException: java.io.FileNotFoundException: Hadoop bin
+   * directory does not exist: <path>\hadoop-common-project
+   * \hadoop-common\target\bin -see https://wiki.apache
+   * .org/hadoop/WindowsProblems. then do build the hadoop dependencies
+   * otherwise mark this test as skip.
+   */
+  @Override
+  public void testWorkingDirectory() throws Exception {
+    super.testWorkingDirectory();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c9e71382/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/oauth2/TestCachedRefreshTokenBasedAccessTokenProvider.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/oauth2/TestCachedRefreshTokenBasedAccessTokenProvider.java b/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/oauth2/TestCachedRefreshTokenBasedAccessTokenProvider.java
index e57d3a9..c044594 100644
--- a/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/oauth2/TestCachedRefreshTokenBasedAccessTokenProvider.java
+++ b/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/oauth2/TestCachedRefreshTokenBasedAccessTokenProvider.java
@@ -20,17 +20,19 @@ package org.apache.hadoop.fs.adl.oauth2;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdfs.web.oauth2.AccessTokenProvider;
-
 import org.junit.Before;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.rules.TestName;
-import static org.junit.Assert.*;
-import static org.mockito.Mockito.*;
 
 import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.OAUTH_CLIENT_ID_KEY;
 import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.OAUTH_REFRESH_URL_KEY;
 import static org.apache.hadoop.hdfs.web.oauth2.ConfRefreshTokenBasedAccessTokenProvider.OAUTH_REFRESH_TOKEN_KEY;
+import static org.junit.Assert.fail;
+import static org.mockito.Mockito.any;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
 
 /**
  * Verify cache behavior of ConfRefreshTokenBasedAccessTokenProvider instances.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c9e71382/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/common/AdlMockWebServer.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/common/AdlMockWebServer.java b/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/common/AdlMockWebServer.java
new file mode 100644
index 0000000..69bb9ae
--- /dev/null
+++ b/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/common/AdlMockWebServer.java
@@ -0,0 +1,116 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+
+package org.apache.hadoop.fs.common;
+
+import com.eclipsesource.json.JsonObject;
+import com.squareup.okhttp.mockwebserver.MockResponse;
+import com.squareup.okhttp.mockwebserver.MockWebServer;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.adl.TestableAdlFileSystem;
+import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
+import org.apache.hadoop.hdfs.web.oauth2.ConfCredentialBasedAccessTokenProvider;
+import org.apache.hadoop.hdfs.web.oauth2.CredentialBasedAccessTokenProvider;
+import org.apache.hadoop.hdfs.web.oauth2.OAuth2Constants;
+import org.junit.After;
+import org.junit.Before;
+
+import java.io.IOException;
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.net.URL;
+
+/**
+ * Mock server to simulate Adls backend calls. This infrastructure is expandable
+ * to override expected server response based on the derived test functionality.
+ * Common functionality to generate token information before request is send to
+ * adls backend is also managed within AdlMockWebServer implementation.
+ */
+public class AdlMockWebServer {
+
+  // Create a MockWebServer. These are lean enough that you can create a new
+  // instance for every unit test.
+  private MockWebServer server = null;
+  private TestableAdlFileSystem fs = null;
+  private int port = 0;
+  private Configuration conf = new Configuration();
+
+  public MockWebServer getMockServer() {
+    return server;
+  }
+
+  public TestableAdlFileSystem getMockAdlFileSystem() {
+    return fs;
+  }
+
+  public int getPort() {
+    return port;
+  }
+
+  public Configuration getConf() {
+    return conf;
+  }
+
+  public static MockResponse getTokenResponse() {
+    JsonObject jsonObject = new JsonObject()
+        .set(OAuth2Constants.EXPIRES_IN, "0987654321")
+        .set("token_type", "bearer").set(OAuth2Constants.ACCESS_TOKEN, "123");
+    MockResponse oauth2Response = new MockResponse();
+    oauth2Response.addHeader("Content-Type", "application/json");
+    oauth2Response.setResponseCode(200);
+    oauth2Response.setBody(jsonObject.toString());
+    return oauth2Response;
+  }
+
+  @Before
+  public void preTestSetup() throws IOException, URISyntaxException {
+
+    server = new MockWebServer();
+    server.enqueue(getTokenResponse());
+
+    // Start the server.
+    server.start();
+
+    // Ask the server for its URL. You'll need this to make HTTP requests.
+    URL baseUrl = server.getUrl("");
+    port = baseUrl.getPort();
+
+    // Exercise your application code, which should make those HTTP requests.
+    // Responses are returned in the same order that they are enqueued.
+    fs = new TestableAdlFileSystem();
+
+    conf.set(HdfsClientConfigKeys.OAUTH_CLIENT_ID_KEY, "MY_CLIENTID");
+    conf.set(HdfsClientConfigKeys.ACCESS_TOKEN_PROVIDER_KEY,
+        ConfCredentialBasedAccessTokenProvider.class.getName());
+    conf.set(HdfsClientConfigKeys.DFS_WEBHDFS_OAUTH_ENABLED_KEY, "true");
+    conf.set(HdfsClientConfigKeys.OAUTH_REFRESH_URL_KEY, "http://localhost:" +
+        port + "/refresh");
+    conf.set(CredentialBasedAccessTokenProvider.OAUTH_CREDENTIAL_KEY,
+        "credential");
+
+    URI uri = new URI("adl://localhost:" + port);
+    fs.initialize(uri, conf);
+  }
+
+  @After
+  public void postTestSetup() throws IOException {
+    fs.close();
+    server.shutdown();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c9e71382/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/common/ExpectedResponse.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/common/ExpectedResponse.java b/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/common/ExpectedResponse.java
new file mode 100644
index 0000000..727417e
--- /dev/null
+++ b/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/common/ExpectedResponse.java
@@ -0,0 +1,72 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+
+package org.apache.hadoop.fs.common;
+
+
+import com.squareup.okhttp.mockwebserver.MockResponse;
+
+import java.util.ArrayList;
+
+/**
+ * Supporting class to hold expected MockResponse object along with parameters
+ * for validation in test methods.
+ */
+public class ExpectedResponse {
+  private MockResponse response;
+  private ArrayList<String> expectedQueryParameters = new ArrayList<String>();
+  private int expectedBodySize;
+  private String httpRequestType;
+
+  public int getExpectedBodySize() {
+    return expectedBodySize;
+  }
+
+  public String getHttpRequestType() {
+    return httpRequestType;
+  }
+
+  public ArrayList<String> getExpectedQueryParameters() {
+    return expectedQueryParameters;
+  }
+
+  public MockResponse getResponse() {
+    return response;
+  }
+
+  ExpectedResponse set(MockResponse mockResponse) {
+    this.response = mockResponse;
+    return this;
+  }
+
+  ExpectedResponse addExpectedQueryParam(String param) {
+    expectedQueryParameters.add(param);
+    return this;
+  }
+
+  ExpectedResponse addExpectedBodySize(int bodySize) {
+    this.expectedBodySize = bodySize;
+    return this;
+  }
+
+  ExpectedResponse addExpectedHttpRequestType(String expectedHttpRequestType) {
+    this.httpRequestType = expectedHttpRequestType;
+    return this;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c9e71382/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/common/TestDataForRead.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/common/TestDataForRead.java b/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/common/TestDataForRead.java
new file mode 100644
index 0000000..150dc6c
--- /dev/null
+++ b/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/common/TestDataForRead.java
@@ -0,0 +1,120 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+
+package org.apache.hadoop.fs.common;
+
+import com.squareup.okhttp.mockwebserver.Dispatcher;
+import com.squareup.okhttp.mockwebserver.MockResponse;
+import com.squareup.okhttp.mockwebserver.RecordedRequest;
+import okio.Buffer;
+import org.apache.hadoop.fs.adl.TestADLResponseData;
+
+import java.util.ArrayList;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+/**
+ * Supporting class for mock test to validate Adls read operation using
+ * BufferManager.java and BatchByteArrayInputStream implementation.
+ */
+public class TestDataForRead {
+
+  private byte[] actualData;
+  private ArrayList<ExpectedResponse> responses;
+  private Dispatcher dispatcher;
+  private int intensityOfTest;
+  private boolean checkOfNoOfCalls;
+  private int expectedNoNetworkCall;
+
+  public TestDataForRead(final byte[] actualData, int expectedNoNetworkCall,
+      int intensityOfTest, boolean checkOfNoOfCalls) {
+    this.checkOfNoOfCalls = checkOfNoOfCalls;
+    this.actualData = actualData;
+    responses = new ArrayList<ExpectedResponse>();
+    this.expectedNoNetworkCall = expectedNoNetworkCall;
+    this.intensityOfTest = intensityOfTest;
+
+    dispatcher = new Dispatcher() {
+      @Override
+      public MockResponse dispatch(RecordedRequest recordedRequest)
+          throws InterruptedException {
+        if (recordedRequest.getPath().equals("/refresh")) {
+          return AdlMockWebServer.getTokenResponse();
+        }
+
+        if (recordedRequest.getRequestLine().contains("op=GETFILESTATUS")) {
+          return new MockResponse().setResponseCode(200).setBody(
+              TestADLResponseData
+                  .getGetFileStatusJSONResponse(actualData.length));
+        }
+
+        if (recordedRequest.getRequestLine().contains("op=OPEN")) {
+          String request = recordedRequest.getRequestLine();
+          int offset = 0;
+          int byteCount = 0;
+
+          Pattern pattern = Pattern.compile("offset=([0-9]+)");
+          Matcher matcher = pattern.matcher(request);
+          if (matcher.find()) {
+            System.out.println(matcher.group(1));
+            offset = Integer.parseInt(matcher.group(1));
+          }
+
+          pattern = Pattern.compile("length=([0-9]+)");
+          matcher = pattern.matcher(request);
+          if (matcher.find()) {
+            System.out.println(matcher.group(1));
+            byteCount = Integer.parseInt(matcher.group(1));
+          }
+
+          Buffer buf = new Buffer();
+          buf.write(actualData, offset, byteCount);
+          return new MockResponse().setResponseCode(200)
+              .setChunkedBody(buf, 4 * 1024 * 1024);
+        }
+
+        return new MockResponse().setBody("NOT SUPPORTED").setResponseCode(501);
+      }
+    };
+  }
+
+  public boolean isCheckOfNoOfCalls() {
+    return checkOfNoOfCalls;
+  }
+
+  public int getExpectedNoNetworkCall() {
+    return expectedNoNetworkCall;
+  }
+
+  public int getIntensityOfTest() {
+    return intensityOfTest;
+  }
+
+  public byte[] getActualData() {
+    return actualData;
+  }
+
+  public ArrayList<ExpectedResponse> getResponses() {
+    return responses;
+  }
+
+  public Dispatcher getDispatcher() {
+    return dispatcher;
+  }
+}


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


[05/39] hadoop git commit: HADOOP-13179. GenericOptionsParser is not thread-safe because commons-cli OptionBuilder is not thread-safe. (Contributed by hongbin ma)

Posted by ae...@apache.org.
HADOOP-13179. GenericOptionsParser is not thread-safe because commons-cli OptionBuilder is not thread-safe. (Contributed by hongbin ma)


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

Branch: refs/heads/HDFS-1312
Commit: 8e8cb4c6ca7e803674b160946ac92274bac0f96d
Parents: 20b13d1
Author: Vinayakumar B <vi...@apache.org>
Authored: Tue Jun 14 15:58:04 2016 +0530
Committer: Vinayakumar B <vi...@apache.org>
Committed: Tue Jun 14 16:01:26 2016 +0530

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


http://git-wip-us.apache.org/repos/asf/hadoop/blob/8e8cb4c6/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/GenericOptionsParser.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/GenericOptionsParser.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/GenericOptionsParser.java
index 17ce730..8a06b1f 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/GenericOptionsParser.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/GenericOptionsParser.java
@@ -208,7 +208,7 @@ public class GenericOptionsParser {
    * Specify properties of each generic option
    */
   @SuppressWarnings("static-access")
-  private static Options buildGeneralOptions(Options opts) {
+  private static synchronized Options buildGeneralOptions(Options opts) {
     Option fs = OptionBuilder.withArgName("local|namenode:port")
     .hasArg()
     .withDescription("specify a namenode")


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


[18/39] hadoop git commit: HDFS-10525. Fix NPE in CacheReplicationMonitor#rescanCachedBlockMap (Xiao Chen via cmccabe)

Posted by ae...@apache.org.
HDFS-10525. Fix NPE in CacheReplicationMonitor#rescanCachedBlockMap (Xiao Chen via cmccabe)


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

Branch: refs/heads/HDFS-1312
Commit: 2ca73445f5c2929d9c2ff4232dca58a63a0570a0
Parents: 2449db5
Author: Colin Patrick Mccabe <cm...@cloudera.com>
Authored: Wed Jun 15 22:47:01 2016 -0700
Committer: Colin Patrick Mccabe <cm...@cloudera.com>
Committed: Wed Jun 15 22:47:01 2016 -0700

----------------------------------------------------------------------
 .../hdfs/server/blockmanagement/CacheReplicationMonitor.java | 8 ++++++++
 1 file changed, 8 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/2ca73445/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/CacheReplicationMonitor.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/CacheReplicationMonitor.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/CacheReplicationMonitor.java
index b4b5e23..ca8d72a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/CacheReplicationMonitor.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/CacheReplicationMonitor.java
@@ -502,6 +502,14 @@ public class CacheReplicationMonitor extends Thread implements Closeable {
         CachedBlock cblock = it.next();
         BlockInfo blockInfo = blockManager.
             getStoredBlock(new Block(cblock.getBlockId()));
+        if (blockInfo == null) {
+          // Cannot find this block on the NameNode, skip this block from
+          // capacity calculation. Later logic will handle this block.
+          LOG.debug("Block {}: cannot be found in block manager and hence"
+              + " skipped from calculation for node {}.", cblock.getBlockId(),
+              dn.getDatanodeUuid());
+          continue;
+        }
         if (blockInfo.getNumBytes() > remaining) {
           LOG.debug("Block {}: removing from PENDING_CACHED for node {} "
                   + "because it cannot fit in remaining cache size {}.",


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


[25/39] hadoop git commit: HADOOP-13241. document s3a better. Contributed by Steve Loughran.

Posted by ae...@apache.org.
HADOOP-13241. document s3a better. Contributed by Steve Loughran.


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

Branch: refs/heads/HDFS-1312
Commit: 127d2c7281917f23bce17afa6098a2d678a16441
Parents: 4aefe11
Author: Chris Nauroth <cn...@apache.org>
Authored: Thu Jun 16 10:05:54 2016 -0700
Committer: Chris Nauroth <cn...@apache.org>
Committed: Thu Jun 16 11:18:02 2016 -0700

----------------------------------------------------------------------
 .../src/main/resources/core-default.xml         |   1 +
 .../src/site/markdown/tools/hadoop-aws/index.md | 308 ++++++++++++++++++-
 2 files changed, 308 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/127d2c72/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml b/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml
index 089a2c1..ec212e7 100644
--- a/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml
+++ b/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml
@@ -845,6 +845,7 @@
 
 <property>
   <name>fs.s3a.path.style.access</name>
+  <value>false</value>
   <description>Enable S3 path style access ie disabling the default virtual hosting behaviour.
     Useful for S3A-compliant storage providers as it removes the need to set up DNS for virtual hosting.
   </description>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/127d2c72/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 79c9349..d677baa 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
@@ -77,9 +77,34 @@ Do not inadvertently share these credentials through means such as
 
 If you do any of these: change your credentials immediately!
 
+### Warning #4: the S3 client provided by Amazon EMR are not from the Apache
+Software foundation, and are only supported by Amazon.
+
+Specifically: on Amazon EMR, s3a is not supported, and amazon recommend
+a different filesystem implementation. If you are using Amazon EMR, follow
+these instructions \u2014and be aware that all issues related to S3 integration
+in EMR can only be addressed by Amazon themselves: please raise your issues
+with them.
 
 ## S3
 
+The `s3://` filesystem is the original S3 store in the Hadoop codebase.
+It implements an inode-style filesystem atop S3, and was written to
+provide scaleability when S3 had significant limits on the size of blobs.
+It is incompatible with any other application's use of data in S3.
+
+It is now deprecated and will be removed in Hadoop 3. Please do not use,
+and migrate off data which is on it.
+
+### Dependencies
+
+* `jets3t` jar
+* `commons-codec` jar
+* `commons-logging` jar
+* `httpclient` jar
+* `httpcore` jar
+* `java-xmlbuilder` jar
+
 ### Authentication properties
 
     <property>
@@ -95,6 +120,42 @@ If you do any of these: change your credentials immediately!
 
 ## S3N
 
+S3N was the first S3 Filesystem client which used "native" S3 objects, hence
+the schema `s3n://`.
+
+### Features
+
+* Directly reads and writes S3 objects.
+* Compatible with standard S3 clients.
+* Supports partitioned uploads for many-GB objects.
+* Available across all Hadoop 2.x releases.
+
+The S3N filesystem client, while widely used, is no longer undergoing
+active maintenance except for emergency security issues. There are
+known bugs, especially: it reads to end of a stream when closing a read;
+this can make `seek()` slow on large files. The reason there has been no
+attempt to fix this is that every upgrade of the Jets3t library, while
+fixing some problems, has unintentionally introduced new ones in either the changed
+Hadoop code, or somewhere in the Jets3t/Httpclient code base.
+The number of defects remained constant, they merely moved around.
+
+By freezing the Jets3t jar version and avoiding changes to the code,
+we reduce the risk of making things worse.
+
+The S3A filesystem client can read all files created by S3N. Accordingly
+it should be used wherever possible.
+
+
+### Dependencies
+
+* `jets3t` jar
+* `commons-codec` jar
+* `commons-logging` jar
+* `httpclient` jar
+* `httpcore` jar
+* `java-xmlbuilder` jar
+
+
 ### Authentication properties
 
     <property>
@@ -176,6 +237,45 @@ If you do any of these: change your credentials immediately!
 ## S3A
 
 
+The S3A filesystem client, prefix `s3a://`, is the S3 client undergoing
+active development and maintenance.
+While this means that there is a bit of instability
+of configuration options and behavior, it also means
+that the code is getting better in terms of reliability, performance,
+monitoring and other features.
+
+### Features
+
+* Directly reads and writes S3 objects.
+* Compatible with standard S3 clients.
+* Can read data created with S3N.
+* Can write data back that is readable by S3N. (Note: excluding encryption).
+* Supports partitioned uploads for many-GB objects.
+* Instrumented with Hadoop metrics.
+* Performance optimized operations, including `seek()` and `readFully()`.
+* Uses Amazon's Java S3 SDK with support for latest S3 features and authentication
+schemes.
+* Supports authentication via: environment variables, Hadoop configuration
+properties, the Hadoop key management store and IAM roles.
+* Supports S3 "Server Side Encryption" for both reading and writing.
+* Supports proxies
+* Test suites includes distcp and suites in downstream projects.
+* Available since Hadoop 2.6; considered production ready in Hadoop 2.7.
+* Actively maintained.
+
+S3A is now the recommended client for working with S3 objects. It is also the
+one where patches for functionality and performance are very welcome.
+
+### Dependencies
+
+* `hadoop-aws` jar.
+* `aws-java-sdk-s3` jar.
+* `aws-java-sdk-core` jar.
+* `aws-java-sdk-kms` jar.
+* `joda-time` jar; use version 2.8.1 or later.
+* `httpclient` jar.
+* Jackson `jackson-core`, `jackson-annotations`, `jackson-databind` jars.
+
 ### Authentication properties
 
     <property>
@@ -333,6 +433,7 @@ this capability.
 
     <property>
       <name>fs.s3a.path.style.access</name>
+      <value>false</value>
       <description>Enable S3 path style access ie disabling the default virtual hosting behaviour.
         Useful for S3A-compliant storage providers as it removes the need to set up DNS for virtual hosting.
       </description>
@@ -432,7 +533,7 @@ this capability.
 
     <property>
       <name>fs.s3a.multiobjectdelete.enable</name>
-      <value>false</value>
+      <value>true</value>
       <description>When enabled, multiple single-object delete requests are replaced by
         a single 'delete multiple objects'-request, reducing the number of requests.
         Beware: legacy S3-compatible object stores might not support this request.
@@ -556,6 +657,211 @@ the available memory. These settings should be tuned to the envisioned
 workflow (some large files, many small ones, ...) and the physical
 limitations of the machine and cluster (memory, network bandwidth).
 
+## Troubleshooting S3A
+
+Common problems working with S3A are
+
+1. Classpath
+1. Authentication
+1. S3 Inconsistency side-effects
+
+Classpath is usually the first problem. For the S3x filesystem clients,
+you need the Hadoop-specific filesystem clients, third party S3 client libraries
+compatible with the Hadoop code, and any dependent libraries compatible with
+Hadoop and the specific JVM.
+
+The classpath must be set up for the process talking to S3: if this is code
+running in the Hadoop cluster, the JARs must be on that classpath. That
+includes `distcp`.
+
+
+### `ClassNotFoundException: org.apache.hadoop.fs.s3a.S3AFileSystem`
+
+(or `org.apache.hadoop.fs.s3native.NativeS3FileSystem`, `org.apache.hadoop.fs.s3.S3FileSystem`).
+
+These are the Hadoop classes, found in the `hadoop-aws` JAR. An exception
+reporting one of these classes is missing means that this JAR is not on
+the classpath.
+
+### `ClassNotFoundException: com.amazonaws.services.s3.AmazonS3Client`
+
+(or other `com.amazonaws` class.)
+`
+This means that one or more of the `aws-*-sdk` JARs are missing. Add them.
+
+### Missing method in AWS class
+
+This can be triggered by incompatibilities between the AWS SDK on the classpath
+and the version which Hadoop was compiled with.
+
+The AWS SDK JARs change their signature enough between releases that the only
+way to safely update the AWS SDK version is to recompile Hadoop against the later
+version.
+
+There's nothing the Hadoop team can do here: if you get this problem, then sorry,
+but you are on your own. The Hadoop developer team did look at using reflection
+to bind to the SDK, but there were too many changes between versions for this
+to work reliably. All it did was postpone version compatibility problems until
+the specific codepaths were executed at runtime \u2014this was actually a backward
+step in terms of fast detection of compatibility problems.
+
+### Missing method in a Jackson class
+
+This is usually caused by version mismatches between Jackson JARs on the
+classpath. All Jackson JARs on the classpath *must* be of the same version.
+
+
+### Authentication failure
+
+One authentication problem is caused by classpath mismatch; see the joda time
+issue above.
+
+Otherwise, the general cause is: you have the wrong credentials \u2014or somehow
+the credentials were not readable on the host attempting to read or write
+the S3 Bucket.
+
+There's not much that Hadoop can do/does for diagnostics here,
+though enabling debug logging for the package `org.apache.hadoop.fs.s3a`
+can help.
+
+There is also some logging in the AWS libraries which provide some extra details.
+In particular, the setting the log `com.amazonaws.auth.AWSCredentialsProviderChain`
+to log at DEBUG level will mean the invidual reasons for the (chained)
+authentication clients to fail will be printed.
+
+Otherwise, try to use the AWS command line tools with the same credentials.
+If you set the environment variables, you can take advantage of S3A's support
+of environment-variable authentication by attempting to use the `hdfs fs` command
+to read or write data on S3. That is: comment out the `fs.s3a` secrets and rely on
+the environment variables.
+
+S3 Frankfurt is a special case. It uses the V4 authentication API.
+
+### Authentication failures running on Java 8u60+
+
+A change in the Java 8 JVM broke some of the `toString()` string generation
+of Joda Time 2.8.0, which stopped the amazon s3 client from being able to
+generate authentication headers suitable for validation by S3.
+
+Fix: make sure that the version of Joda Time is 2.8.1 or later.
+
+## Visible S3 Inconsistency
+
+Amazon S3 is *an eventually consistent object store*. That is: not a filesystem.
+
+It offers read-after-create consistency: a newly created file is immediately
+visible. Except, there is a small quirk: a negative GET may be cached, such
+that even if an object is immediately created, the fact that there "wasn't"
+an object is still remembered.
+
+That means the following sequence on its own will be consistent
+```
+touch(path) -> getFileStatus(path)
+```
+
+But this sequence *may* be inconsistent.
+
+```
+getFileStatus(path) -> touch(path) -> getFileStatus(path)
+```
+
+A common source of visible inconsistencies is that the S3 metadata
+database \u2014the part of S3 which serves list requests\u2014 is updated asynchronously.
+Newly added or deleted files may not be visible in the index, even though direct
+operations on the object (`HEAD` and `GET`) succeed.
+
+In S3A, that means the `getFileStatus()` and `open()` operations are more likely
+to be consistent with the state of the object store than any directory list
+operations (`listStatus()`, `listFiles()`, `listLocatedStatus()`,
+`listStatusIterator()`).
+
+
+### `FileNotFoundException` even though the file was just written.
+
+This can be a sign of consistency problems. It may also surface if there is some
+asynchronous file write operation still in progress in the client: the operation
+has returned, but the write has not yet completed. While the S3A client code
+does block during the `close()` operation, we suspect that asynchronous writes
+may be taking place somewhere in the stack \u2014this could explain why parallel tests
+fail more often than serialized tests.
+
+### File not found in a directory listing, even though `getFileStatus()` finds it
+
+(Similarly: deleted file found in listing, though `getFileStatus()` reports
+that it is not there)
+
+This is a visible sign of updates to the metadata server lagging
+behind the state of the underlying filesystem.
+
+
+### File not visible/saved
+
+The files in an object store are not visible until the write has been completed.
+In-progress writes are simply saved to a local file/cached in RAM and only uploaded.
+at the end of a write operation. If a process terminated unexpectedly, or failed
+to call the `close()` method on an output stream, the pending data will have
+been lost.
+
+### File `flush()` and `hflush()` calls do not save data to S3A
+
+Again, this is due to the fact that the data is cached locally until the
+`close()` operation. The S3A filesystem cannot be used as a store of data
+if it is required that the data is persisted durably after every
+`flush()/hflush()` call. This includes resilient logging, HBase-style journalling
+and the like. The standard strategy here is to save to HDFS and then copy to S3.
+
+### Other issues
+
+*Performance slow*
+
+S3 is slower to read data than HDFS, even on virtual clusters running on
+Amazon EC2.
+
+* HDFS replicates data for faster query performance
+* HDFS stores the data on the local hard disks, avoiding network traffic
+ if the code can be executed on that host. As EC2 hosts often have their
+ network bandwidth throttled, this can make a tangible difference.
+* HDFS is significantly faster for many "metadata" operations: listing
+the contents of a directory, calling `getFileStatus()` on path,
+creating or deleting directories.
+* On HDFS, Directory renames and deletes are `O(1)` operations. On
+S3 renaming is a very expensive `O(data)` operation which may fail partway through
+in which case the final state depends on where the copy+ delete sequence was when it failed.
+All the objects are copied, then the original set of objects are deleted, so
+a failure should not lose data \u2014it may result in duplicate datasets.
+* Because the write only begins on a `close()` operation, it may be in the final
+phase of a process where the write starts \u2014this can take so long that some things
+can actually time out.
+
+The slow performance of `rename()` surfaces during the commit phase of work,
+including
+
+* The MapReduce FileOutputCommitter.
+* DistCp's rename after copy operation.
+
+Both these operations can be significantly slower when S3 is the destination
+compared to HDFS or other "real" filesystem.
+
+*Improving S3 load-balancing behavior*
+
+Amazon S3 uses a set of front-end servers to provide access to the underlying data.
+The choice of which front-end server to use is handled via load-balancing DNS
+service: when the IP address of an S3 bucket is looked up, the choice of which
+IP address to return to the client is made based on the the current load
+of the front-end servers.
+
+Over time, the load across the front-end changes, so those servers considered
+"lightly loaded" will change. If the DNS value is cached for any length of time,
+your application may end up talking to an overloaded server. Or, in the case
+of failures, trying to talk to a server that is no longer there.
+
+And by default, for historical security reasons in the era of applets,
+the DNS TTL of a JVM is "infinity".
+
+To work with AWS better, set the DNS time-to-live of an application which
+works with S3 to something lower. See [AWS documentation](http://docs.aws.amazon.com/AWSSdkDocsJava/latest/DeveloperGuide/java-dg-jvm-ttl.html).
+
+
 ## Testing the S3 filesystem clients
 
 Due to eventual consistency, tests may fail without reason. Transient


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


[21/39] hadoop git commit: HADOOP-12875. [Azure Data Lake] Support for contract test and unit test cases. Contributed by Vishwajeet Dusane.

Posted by ae...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/c9e71382/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/hdfs/web/TestAdlRead.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/hdfs/web/TestAdlRead.java b/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/hdfs/web/TestAdlRead.java
new file mode 100644
index 0000000..60904d0
--- /dev/null
+++ b/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/hdfs/web/TestAdlRead.java
@@ -0,0 +1,205 @@
+/*
+ * 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.web;
+
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.adl.TestADLResponseData;
+import org.apache.hadoop.fs.common.AdlMockWebServer;
+import org.apache.hadoop.fs.common.TestDataForRead;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.Timeout;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import java.io.ByteArrayInputStream;
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Random;
+
+/**
+ * This class is responsible for stress positional reads vs number of network
+ * calls required by to fetch the amount of data. Test does ensure the data
+ * integrity and order of the data is maintained. This tests are meant to test
+ * BufferManager.java and BatchByteArrayInputStream implementation.
+ */
+@RunWith(Parameterized.class)
+public class TestAdlRead extends AdlMockWebServer {
+
+  // Keeping timeout of 1 hour to ensure the test does complete and should
+  // not terminate due to high backend latency.
+  @Rule
+  public Timeout globalTimeout = new Timeout(60 * 60000);
+  private TestDataForRead testData;
+
+  public TestAdlRead(TestDataForRead testData) {
+    this.testData = testData;
+    getConf().set("adl.feature.override.readahead.max.buffersize", "8192");
+    getConf().set("adl.feature.override.readahead.max.concurrent.connection",
+        "1");
+  }
+
+  @Parameterized.Parameters(name = "{index}")
+  public static Collection testDataForReadOperation() {
+    return Arrays.asList(new Object[][] {
+
+        //--------------------------
+        // Test Data
+        //--------------------------
+        {new TestDataForRead("Hello World".getBytes(), 3, 1000, true)},
+        {new TestDataForRead(
+            ("the problem you appear to be wrestling with is that this doesn't "
+                + "display very well. ").getBytes(), 3, 1000, true)},
+        {new TestDataForRead(
+            ("Chinese Indonesians (Indonesian: Orang Tionghoa-Indonesia; "
+                + "Chinese: "
+                + "trad ???????, simp ???????, pin Y\ufffdnd\ufffdn\ufffdx?y\ufffd Hu\ufffdr\ufffdn), are "
+                + "Indonesians descended from various Chinese ethnic groups, "
+                + "particularly Han.").getBytes(), 3, 1000, true)},
+        {new TestDataForRead(
+            TestADLResponseData.getRandomByteArrayData(5 * 1024), 3, 1000,
+            true)}, {new TestDataForRead(
+        TestADLResponseData.getRandomByteArrayData(1 * 1024), 3, 50, true)},
+        {new TestDataForRead(
+            TestADLResponseData.getRandomByteArrayData(8 * 1024), 3, 10, true)},
+        {new TestDataForRead(
+            TestADLResponseData.getRandomByteArrayData(32 * 1024), 6, 10,
+            false)}, {new TestDataForRead(
+        TestADLResponseData.getRandomByteArrayData(48 * 1024), 8, 10, false)}});
+  }
+
+  @After
+  @Before
+  public void cleanReadBuffer() {
+    BufferManager.getInstance().clear();
+  }
+
+  @Test
+  public void testEntireBytes() throws IOException, InterruptedException {
+    getMockServer().setDispatcher(testData.getDispatcher());
+    FSDataInputStream in = getMockAdlFileSystem().open(new Path("/test"));
+    byte[] expectedData = new byte[testData.getActualData().length];
+    Assert.assertEquals(in.read(expectedData), expectedData.length);
+    Assert.assertArrayEquals(expectedData, testData.getActualData());
+    in.close();
+    if (testData.isCheckOfNoOfCalls()) {
+      Assert.assertEquals(testData.getExpectedNoNetworkCall(),
+          getMockServer().getRequestCount());
+    }
+  }
+
+  @Test
+  public void testSeekOperation() throws IOException, InterruptedException {
+    getMockServer().setDispatcher(testData.getDispatcher());
+    FSDataInputStream in = getMockAdlFileSystem().open(new Path("/test"));
+    Random random = new Random();
+    for (int i = 0; i < 1000; ++i) {
+      int position = random.nextInt(testData.getActualData().length);
+      in.seek(position);
+      Assert.assertEquals(in.getPos(), position);
+      Assert.assertEquals(in.read(), testData.getActualData()[position] & 0xFF);
+    }
+    in.close();
+    if (testData.isCheckOfNoOfCalls()) {
+      Assert.assertEquals(testData.getExpectedNoNetworkCall(),
+          getMockServer().getRequestCount());
+    }
+  }
+
+  @Test
+  public void testReadServerCalls() throws IOException, InterruptedException {
+    getMockServer().setDispatcher(testData.getDispatcher());
+    FSDataInputStream in = getMockAdlFileSystem().open(new Path("/test"));
+    byte[] expectedData = new byte[testData.getActualData().length];
+    in.readFully(expectedData);
+    Assert.assertArrayEquals(expectedData, testData.getActualData());
+    Assert.assertEquals(testData.getExpectedNoNetworkCall(),
+        getMockServer().getRequestCount());
+    in.close();
+  }
+
+  @Test
+  public void testReadFully() throws IOException, InterruptedException {
+    getMockServer().setDispatcher(testData.getDispatcher());
+    FSDataInputStream in = getMockAdlFileSystem().open(new Path("/test"));
+    byte[] expectedData = new byte[testData.getActualData().length];
+    in.readFully(expectedData);
+    Assert.assertArrayEquals(expectedData, testData.getActualData());
+
+    in.readFully(0, expectedData);
+    Assert.assertArrayEquals(expectedData, testData.getActualData());
+
+    in.seek(0);
+    in.readFully(expectedData, 0, expectedData.length);
+    Assert.assertArrayEquals(expectedData, testData.getActualData());
+    in.close();
+
+    if (testData.isCheckOfNoOfCalls()) {
+      Assert.assertEquals(testData.getExpectedNoNetworkCall(),
+          getMockServer().getRequestCount());
+    }
+  }
+
+  @Test
+  public void testRandomPositionalReadUsingReadFully()
+      throws IOException, InterruptedException {
+    getMockServer().setDispatcher(testData.getDispatcher());
+    FSDataInputStream in = getMockAdlFileSystem().open(new Path("/test"));
+    ByteArrayInputStream actualData = new ByteArrayInputStream(
+        testData.getActualData());
+    Random random = new Random();
+    for (int i = 0; i < testData.getIntensityOfTest(); ++i) {
+      int offset = random.nextInt(testData.getActualData().length);
+      int length = testData.getActualData().length - offset;
+      byte[] expectedData = new byte[length];
+      byte[] actualDataSubset = new byte[length];
+      actualData.reset();
+      actualData.skip(offset);
+      actualData.read(actualDataSubset, 0, length);
+
+      in.readFully(offset, expectedData, 0, length);
+      Assert.assertArrayEquals(expectedData, actualDataSubset);
+    }
+
+    for (int i = 0; i < testData.getIntensityOfTest(); ++i) {
+      int offset = random.nextInt(testData.getActualData().length);
+      int length = random.nextInt(testData.getActualData().length - offset);
+      byte[] expectedData = new byte[length];
+      byte[] actualDataSubset = new byte[length];
+      actualData.reset();
+      actualData.skip(offset);
+      actualData.read(actualDataSubset, 0, length);
+
+      in.readFully(offset, expectedData, 0, length);
+      Assert.assertArrayEquals(expectedData, actualDataSubset);
+    }
+
+    in.close();
+    if (testData.isCheckOfNoOfCalls()) {
+      Assert.assertEquals(testData.getExpectedNoNetworkCall(),
+          getMockServer().getRequestCount());
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c9e71382/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/hdfs/web/TestConcurrentDataReadOperations.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/hdfs/web/TestConcurrentDataReadOperations.java b/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/hdfs/web/TestConcurrentDataReadOperations.java
new file mode 100644
index 0000000..1194336
--- /dev/null
+++ b/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/hdfs/web/TestConcurrentDataReadOperations.java
@@ -0,0 +1,306 @@
+/*
+ * 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.web;
+
+import com.squareup.okhttp.mockwebserver.Dispatcher;
+import com.squareup.okhttp.mockwebserver.MockResponse;
+import com.squareup.okhttp.mockwebserver.RecordedRequest;
+import okio.Buffer;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.adl.TestADLResponseData;
+import org.apache.hadoop.fs.common.AdlMockWebServer;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import java.io.ByteArrayInputStream;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Random;
+import java.util.UUID;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+/**
+ * This class is responsible for testing multiple threads trying to access same
+ * or multiple files from the offset. This tests are meant to test
+ * BufferManager.java and BatchByteArrayInputStream implementation.
+ */
+@RunWith(Parameterized.class)
+public class TestConcurrentDataReadOperations extends AdlMockWebServer {
+
+  private static FSDataInputStream commonHandle = null;
+  private static Object lock = new Object();
+  private int concurrencyLevel;
+
+  public TestConcurrentDataReadOperations(int concurrencyLevel) {
+    this.concurrencyLevel = concurrencyLevel;
+    getConf().set("adl.feature.override.readahead.max.buffersize", "102400");
+    getConf().set("adl.feature.override.readahead.max.concurrent.connection",
+        "1");
+  }
+
+  @Parameterized.Parameters(name = "{index}")
+  public static Collection testDataNumberOfConcurrentRun() {
+    return Arrays.asList(new Object[][] {{1}, {2}, {3}, {4}, {5}});
+  }
+
+  public static byte[] getRandomByteArrayData(int size) {
+    byte[] b = new byte[size];
+    Random rand = new Random();
+    rand.nextBytes(b);
+    return b;
+  }
+
+  private void setDispatcher(final ArrayList<CreateTestData> testData) {
+    getMockServer().setDispatcher(new Dispatcher() {
+      @Override
+      public MockResponse dispatch(RecordedRequest recordedRequest)
+          throws InterruptedException {
+        if (recordedRequest.getPath().equals("/refresh")) {
+          return AdlMockWebServer.getTokenResponse();
+        }
+
+        CreateTestData currentRequest = null;
+        for (CreateTestData local : testData) {
+          if (recordedRequest.getPath().contains(local.path.toString())) {
+            currentRequest = local;
+            break;
+          }
+        }
+
+        if (currentRequest == null) {
+          new MockResponse().setBody("Request data not found")
+              .setResponseCode(501);
+        }
+
+        if (recordedRequest.getRequestLine().contains("op=GETFILESTATUS")) {
+          return new MockResponse().setResponseCode(200).setBody(
+              TestADLResponseData
+                  .getGetFileStatusJSONResponse(currentRequest.data.length));
+        }
+
+        if (recordedRequest.getRequestLine().contains("op=OPEN")) {
+          String request = recordedRequest.getRequestLine();
+          int offset = 0;
+          int byteCount = 0;
+
+          Pattern pattern = Pattern.compile("offset=([0-9]+)");
+          Matcher matcher = pattern.matcher(request);
+          if (matcher.find()) {
+            System.out.println(matcher.group(1));
+            offset = Integer.parseInt(matcher.group(1));
+          }
+
+          pattern = Pattern.compile("length=([0-9]+)");
+          matcher = pattern.matcher(request);
+          if (matcher.find()) {
+            System.out.println(matcher.group(1));
+            byteCount = Integer.parseInt(matcher.group(1));
+          }
+
+          Buffer buf = new Buffer();
+          buf.write(currentRequest.data, offset, byteCount);
+          return new MockResponse().setResponseCode(200)
+              .setChunkedBody(buf, 4 * 1024 * 1024);
+        }
+
+        return new MockResponse().setBody("NOT SUPPORTED").setResponseCode(501);
+      }
+    });
+  }
+
+  @Before
+  public void resetHandle() {
+    commonHandle = null;
+  }
+
+  @Test
+  public void testParallelReadOnDifferentStreams()
+      throws IOException, InterruptedException, ExecutionException {
+
+    ArrayList<CreateTestData> createTestData = new ArrayList<CreateTestData>();
+
+    Random random = new Random();
+
+    for (int i = 0; i < concurrencyLevel; i++) {
+      CreateTestData testData = new CreateTestData();
+      testData
+          .set(new Path("/test/concurrentRead/" + UUID.randomUUID().toString()),
+              getRandomByteArrayData(random.nextInt(1 * 1024 * 1024)));
+      createTestData.add(testData);
+    }
+
+    setDispatcher(createTestData);
+
+    ArrayList<ReadTestData> readTestData = new ArrayList<ReadTestData>();
+    for (CreateTestData local : createTestData) {
+      ReadTestData localReadData = new ReadTestData();
+      localReadData.set(local.path, local.data, 0);
+      readTestData.add(localReadData);
+    }
+
+    runReadTest(readTestData, false);
+  }
+
+  @Test
+  public void testParallelReadOnSameStreams()
+      throws IOException, InterruptedException, ExecutionException {
+    ArrayList<CreateTestData> createTestData = new ArrayList<CreateTestData>();
+
+    Random random = new Random();
+
+    for (int i = 0; i < 1; i++) {
+      CreateTestData testData = new CreateTestData();
+      testData
+          .set(new Path("/test/concurrentRead/" + UUID.randomUUID().toString()),
+              getRandomByteArrayData(1024 * 1024));
+      createTestData.add(testData);
+    }
+
+    setDispatcher(createTestData);
+
+    ArrayList<ReadTestData> readTestData = new ArrayList<ReadTestData>();
+    ByteArrayInputStream buffered = new ByteArrayInputStream(
+        createTestData.get(0).data);
+
+    ReadTestData readInitially = new ReadTestData();
+    byte[] initialData = new byte[1024 * 1024];
+    buffered.read(initialData);
+
+    readInitially.set(createTestData.get(0).path, initialData, 0);
+    readTestData.add(readInitially);
+    runReadTest(readTestData, false);
+
+    readTestData.clear();
+
+    for (int i = 0; i < concurrencyLevel * 5; i++) {
+      ReadTestData localReadData = new ReadTestData();
+      int offset = random.nextInt((1024 * 1024)-1);
+      int length = 1024 * 1024 - offset;
+      byte[] expectedData = new byte[length];
+      buffered.reset();
+      buffered.skip(offset);
+      buffered.read(expectedData);
+      localReadData.set(createTestData.get(0).path, expectedData, offset);
+      readTestData.add(localReadData);
+    }
+
+    runReadTest(readTestData, true);
+  }
+
+  void runReadTest(ArrayList<ReadTestData> testData, boolean useSameStream)
+      throws InterruptedException, ExecutionException {
+
+    ExecutorService executor = Executors.newFixedThreadPool(testData.size());
+    Future[] subtasks = new Future[testData.size()];
+
+    for (int i = 0; i < testData.size(); i++) {
+      subtasks[i] = executor.submit(
+          new ReadConcurrentRunnable(testData.get(i).data, testData.get(i).path,
+              testData.get(i).offset, useSameStream));
+    }
+
+    executor.shutdown();
+
+    // wait until all tasks are finished
+    executor.awaitTermination(120, TimeUnit.SECONDS);
+
+    for (int i = 0; i < testData.size(); ++i) {
+      Assert.assertTrue((Boolean) subtasks[i].get());
+    }
+  }
+
+  class ReadTestData {
+    private Path path;
+    private byte[] data;
+    private int offset;
+
+    public void set(Path filePath, byte[] dataToBeRead, int fromOffset) {
+      this.path = filePath;
+      this.data = dataToBeRead;
+      this.offset = fromOffset;
+    }
+  }
+
+  class CreateTestData {
+    private Path path;
+    private byte[] data;
+
+    public void set(Path filePath, byte[] dataToBeWritten) {
+      this.path = filePath;
+      this.data = dataToBeWritten;
+    }
+  }
+
+  class ReadConcurrentRunnable implements Callable<Boolean> {
+    private Path path;
+    private int offset;
+    private byte[] expectedData;
+    private boolean useSameStream;
+
+    public ReadConcurrentRunnable(byte[] expectedData, Path path, int offset,
+        boolean useSameStream) {
+      this.path = path;
+      this.offset = offset;
+      this.expectedData = expectedData;
+      this.useSameStream = useSameStream;
+    }
+
+    public Boolean call() throws IOException {
+      try {
+        FSDataInputStream in;
+        if (useSameStream) {
+          synchronized (lock) {
+            if (commonHandle == null) {
+              commonHandle = getMockAdlFileSystem().open(path);
+            }
+            in = commonHandle;
+          }
+        } else {
+          in = getMockAdlFileSystem().open(path);
+        }
+
+        byte[] actualData = new byte[expectedData.length];
+        in.readFully(offset, actualData);
+        Assert.assertArrayEquals("Path :" + path.toString() + " did not match.",
+            expectedData, actualData);
+        if (!useSameStream) {
+          in.close();
+        }
+      } catch (IOException e) {
+        e.printStackTrace();
+        return false;
+      }
+      return true;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c9e71382/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/hdfs/web/TestConfigurationSetting.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/hdfs/web/TestConfigurationSetting.java b/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/hdfs/web/TestConfigurationSetting.java
new file mode 100644
index 0000000..9a54ec2
--- /dev/null
+++ b/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/hdfs/web/TestConfigurationSetting.java
@@ -0,0 +1,112 @@
+/*
+ * 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.web;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.adl.TestableAdlFileSystem;
+import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
+import org.apache.hadoop.hdfs.web.oauth2.ConfCredentialBasedAccessTokenProvider;
+import org.apache.hadoop.hdfs.web.oauth2.CredentialBasedAccessTokenProvider;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.net.URI;
+import java.net.URISyntaxException;
+
+/**
+ * This class is responsible for testing adl file system required configuration
+ * and feature set keys.
+ */
+public class TestConfigurationSetting {
+
+  @Test
+  public void testAllConfiguration() throws URISyntaxException, IOException {
+    TestableAdlFileSystem fs = new TestableAdlFileSystem();
+    Configuration conf = new Configuration();
+    conf.set(HdfsClientConfigKeys.OAUTH_REFRESH_URL_KEY,
+        "http://localhost:1111/refresh");
+    conf.set(CredentialBasedAccessTokenProvider.OAUTH_CREDENTIAL_KEY,
+        "credential");
+    conf.set(HdfsClientConfigKeys.OAUTH_CLIENT_ID_KEY, "MY_CLIENTID");
+    conf.set(HdfsClientConfigKeys.ACCESS_TOKEN_PROVIDER_KEY,
+        ConfCredentialBasedAccessTokenProvider.class.getName());
+    conf.set(HdfsClientConfigKeys.DFS_WEBHDFS_OAUTH_ENABLED_KEY, "true");
+
+    URI uri = new URI("adl://localhost:1234");
+    fs.initialize(uri, conf);
+
+    // Default setting check
+    Assert.assertEquals(true, fs.isFeatureRedirectOff());
+    Assert.assertEquals(true, fs.isFeatureGetBlockLocationLocallyBundled());
+    Assert.assertEquals(true, fs.isFeatureConcurrentReadWithReadAhead());
+    Assert.assertEquals(false, fs.isOverrideOwnerFeatureOn());
+    Assert.assertEquals(8 * 1024 * 1024, fs.getMaxBufferSize());
+    Assert.assertEquals(2, fs.getMaxConcurrentConnection());
+
+    fs.close();
+
+    // Configuration toggle check
+    conf.set("adl.feature.override.redirection.off", "false");
+    fs.initialize(uri, conf);
+    Assert.assertEquals(false, fs.isFeatureRedirectOff());
+    fs.close();
+    conf.set("adl.feature.override.redirection.off", "true");
+    fs.initialize(uri, conf);
+    Assert.assertEquals(true, fs.isFeatureRedirectOff());
+    fs.close();
+
+    conf.set("adl.feature.override.getblocklocation.locally.bundled", "false");
+    fs.initialize(uri, conf);
+    Assert.assertEquals(false, fs.isFeatureGetBlockLocationLocallyBundled());
+    fs.close();
+    conf.set("adl.feature.override.getblocklocation.locally.bundled", "true");
+    fs.initialize(uri, conf);
+    Assert.assertEquals(true, fs.isFeatureGetBlockLocationLocallyBundled());
+    fs.close();
+
+    conf.set("adl.feature.override.readahead", "false");
+    fs.initialize(uri, conf);
+    Assert.assertEquals(false, fs.isFeatureConcurrentReadWithReadAhead());
+    fs.close();
+    conf.set("adl.feature.override.readahead", "true");
+    fs.initialize(uri, conf);
+    Assert.assertEquals(true, fs.isFeatureConcurrentReadWithReadAhead());
+    fs.close();
+
+    conf.set("adl.feature.override.readahead.max.buffersize", "101");
+    fs.initialize(uri, conf);
+    Assert.assertEquals(101, fs.getMaxBufferSize());
+    fs.close();
+    conf.set("adl.feature.override.readahead.max.buffersize", "12134565");
+    fs.initialize(uri, conf);
+    Assert.assertEquals(12134565, fs.getMaxBufferSize());
+    fs.close();
+
+    conf.set("adl.debug.override.localuserasfileowner", "true");
+    fs.initialize(uri, conf);
+    Assert.assertEquals(true, fs.isOverrideOwnerFeatureOn());
+    fs.close();
+    conf.set("adl.debug.override.localuserasfileowner", "false");
+    fs.initialize(uri, conf);
+    Assert.assertEquals(false, fs.isOverrideOwnerFeatureOn());
+    fs.close();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c9e71382/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/hdfs/web/TestSplitSizeCalculation.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/hdfs/web/TestSplitSizeCalculation.java b/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/hdfs/web/TestSplitSizeCalculation.java
new file mode 100644
index 0000000..42a4701
--- /dev/null
+++ b/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/hdfs/web/TestSplitSizeCalculation.java
@@ -0,0 +1,123 @@
+/*
+ * 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.web;
+
+import com.squareup.okhttp.mockwebserver.MockResponse;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.adl.TestADLResponseData;
+import org.apache.hadoop.fs.common.AdlMockWebServer;
+import org.apache.hadoop.hdfs.web.PrivateAzureDataLakeFileSystem.BatchByteArrayInputStream;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.net.URISyntaxException;
+import java.net.URL;
+
+/**
+ * This class is responsible for testing split size calculation during
+ * read ahead buffer initiation based on the data size and configuration
+ * initialization.
+ */
+public class TestSplitSizeCalculation extends AdlMockWebServer {
+
+  @Test
+  public void testSplitSizeCalculations()
+      throws URISyntaxException, IOException {
+
+    getMockServer().enqueue(new MockResponse().setResponseCode(200).setBody(
+        TestADLResponseData.getGetFileStatusJSONResponse(128 * 1024 * 1024)));
+    getMockServer().enqueue(new MockResponse().setResponseCode(200).setBody(
+        TestADLResponseData.getGetFileStatusJSONResponse(128 * 1024 * 1024)));
+    getMockServer().enqueue(new MockResponse().setResponseCode(200).setBody(
+        TestADLResponseData.getGetFileStatusJSONResponse(128 * 1024 * 1024)));
+    getMockServer().enqueue(new MockResponse().setResponseCode(200).setBody(
+        TestADLResponseData.getGetFileStatusJSONResponse(128 * 1024 * 1024)));
+
+    URL url = getMockServer().getUrl("");
+
+    BatchByteArrayInputStream stream = getMockAdlFileSystem()
+        .new BatchByteArrayInputStream(url,
+        new Path("/test1/test2"), 16 * 1024 * 1024, 4);
+    Assert.assertEquals(1, stream.getSplitSize(1 * 1024 * 1024));
+    Assert.assertEquals(1, stream.getSplitSize(2 * 1024 * 1024));
+    Assert.assertEquals(1, stream.getSplitSize(3 * 1024 * 1024));
+    Assert.assertEquals(1, stream.getSplitSize(4 * 1024 * 1024));
+    Assert.assertEquals(1, stream.getSplitSize(5 * 1024 * 1024));
+    Assert.assertEquals(1, stream.getSplitSize(6 * 1024 * 1024));
+    Assert.assertEquals(1, stream.getSplitSize(7 * 1024 * 1024));
+    Assert.assertEquals(2, stream.getSplitSize(8 * 1024 * 1024));
+    Assert.assertEquals(4, stream.getSplitSize(16 * 1024 * 1024));
+    Assert.assertEquals(3, stream.getSplitSize(12 * 1024 * 1024));
+    Assert.assertEquals(4, stream.getSplitSize(102 * 1024 * 1024));
+    Assert.assertEquals(1, stream.getSplitSize(102));
+    stream.close();
+
+    stream = getMockAdlFileSystem().new BatchByteArrayInputStream(url,
+        new Path("/test1/test2"), 4 * 1024 * 1024, 4);
+    Assert.assertEquals(1, stream.getSplitSize(1 * 1024 * 1024));
+    Assert.assertEquals(1, stream.getSplitSize(2 * 1024 * 1024));
+    Assert.assertEquals(1, stream.getSplitSize(3 * 1024 * 1024));
+    Assert.assertEquals(1, stream.getSplitSize(4 * 1024 * 1024));
+    Assert.assertEquals(1, stream.getSplitSize(5 * 1024 * 1024));
+    Assert.assertEquals(1, stream.getSplitSize(8 * 1024 * 1024));
+    Assert.assertEquals(1, stream.getSplitSize(5 * 1024 * 1024));
+    Assert.assertEquals(1, stream.getSplitSize(6 * 1024 * 1024));
+    Assert.assertEquals(1, stream.getSplitSize(7 * 1024 * 1024));
+    Assert.assertEquals(1, stream.getSplitSize(16 * 1024 * 1024));
+    Assert.assertEquals(1, stream.getSplitSize(12 * 1024 * 1024));
+    Assert.assertEquals(1, stream.getSplitSize(102 * 1024 * 1024));
+    Assert.assertEquals(1, stream.getSplitSize(102));
+    stream.close();
+
+    stream = getMockAdlFileSystem().new BatchByteArrayInputStream(url,
+        new Path("/test1/test2"), 16 * 1024 * 1024, 2);
+    Assert.assertEquals(1, stream.getSplitSize(1 * 1024 * 1024));
+    Assert.assertEquals(1, stream.getSplitSize(2 * 1024 * 1024));
+    Assert.assertEquals(1, stream.getSplitSize(3 * 1024 * 1024));
+    Assert.assertEquals(1, stream.getSplitSize(4 * 1024 * 1024));
+    Assert.assertEquals(1, stream.getSplitSize(5 * 1024 * 1024));
+    Assert.assertEquals(1, stream.getSplitSize(5 * 1024 * 1024));
+    Assert.assertEquals(1, stream.getSplitSize(6 * 1024 * 1024));
+    Assert.assertEquals(1, stream.getSplitSize(7 * 1024 * 1024));
+    Assert.assertEquals(2, stream.getSplitSize(8 * 1024 * 1024));
+    Assert.assertEquals(2, stream.getSplitSize(16 * 1024 * 1024));
+    Assert.assertEquals(2, stream.getSplitSize(12 * 1024 * 1024));
+    Assert.assertEquals(2, stream.getSplitSize(102 * 1024 * 1024));
+    Assert.assertEquals(1, stream.getSplitSize(102));
+    stream.close();
+
+    stream = getMockAdlFileSystem().new BatchByteArrayInputStream(url,
+        new Path("/test1/test2"), 8 * 1024 * 1024, 2);
+    Assert.assertEquals(1, stream.getSplitSize(1 * 1024 * 1024));
+    Assert.assertEquals(1, stream.getSplitSize(2 * 1024 * 1024));
+    Assert.assertEquals(1, stream.getSplitSize(3 * 1024 * 1024));
+    Assert.assertEquals(1, stream.getSplitSize(4 * 1024 * 1024));
+    Assert.assertEquals(1, stream.getSplitSize(5 * 1024 * 1024));
+    Assert.assertEquals(1, stream.getSplitSize(6 * 1024 * 1024));
+    Assert.assertEquals(1, stream.getSplitSize(7 * 1024 * 1024));
+    Assert.assertEquals(2, stream.getSplitSize(8 * 1024 * 1024));
+    Assert.assertEquals(2, stream.getSplitSize(16 * 1024 * 1024));
+    Assert.assertEquals(2, stream.getSplitSize(12 * 1024 * 1024));
+    Assert.assertEquals(2, stream.getSplitSize(102 * 1024 * 1024));
+    Assert.assertEquals(1, stream.getSplitSize(102));
+    stream.close();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c9e71382/hadoop-tools/hadoop-azure-datalake/src/test/resources/adls.xml
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure-datalake/src/test/resources/adls.xml b/hadoop-tools/hadoop-azure-datalake/src/test/resources/adls.xml
new file mode 100644
index 0000000..f72de16
--- /dev/null
+++ b/hadoop-tools/hadoop-azure-datalake/src/test/resources/adls.xml
@@ -0,0 +1,139 @@
+<!--
+  Licensed under the Apache License, Version 2.0 (the "License");
+  you may not use this file except in compliance with the License.
+  You may obtain a copy of the License at
+
+    http://www.apache.org/licenses/LICENSE-2.0
+
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License. See accompanying LICENSE file.
+-->
+<configuration>
+  <property>
+    <name>fs.contract.test.root-tests-enabled</name>
+    <value>true</value>
+  </property>
+
+  <property>
+    <name>fs.contract.test.supports-concat</name>
+    <value>true</value>
+  </property>
+
+  <property>
+    <name>fs.contract.rename-returns-false-if-source-missing</name>
+    <value>true</value>
+  </property>
+
+  <property>
+    <name>fs.contract.test.random-seek-count</name>
+    <value>10</value>
+  </property>
+
+  <property>
+    <name>fs.contract.is-case-sensitive</name>
+    <value>true</value>
+  </property>
+
+  <property>
+    <name>fs.contract.rename-returns-true-if-dest-exists</name>
+    <value>true</value>
+  </property>
+
+  <property>
+    <name>fs.contract.rename-returns-true-if-source-missing</name>
+    <value>true</value>
+  </property>
+
+  <property>
+    <name>fs.contract.rename-creates-dest-dirs</name>
+    <value>true</value>
+  </property>
+
+  <property>
+    <name>fs.contract.rename-remove-dest-if-empty-dir</name>
+    <value>true</value>
+  </property>
+
+  <property>
+    <name>fs.contract.supports-settimes</name>
+    <value>true</value>
+  </property>
+
+  <property>
+    <name>fs.contract.supports-append</name>
+    <value>true</value>
+  </property>
+
+  <property>
+    <name>fs.contract.supports-atomic-directory-delete</name>
+    <value>true</value>
+  </property>
+
+  <property>
+    <name>fs.contract.supports-atomic-rename</name>
+    <value>true</value>
+  </property>
+
+  <property>
+    <name>fs.contract.supports-block-locality</name>
+    <value>true</value>
+  </property>
+
+  <property>
+    <name>fs.contract.supports-concat</name>
+    <value>true</value>
+  </property>
+
+  <property>
+    <name>fs.contract.supports-seek</name>
+    <value>true</value>
+  </property>
+
+  <property>
+    <name>fs.contract.supports-seek-on-closed-file</name>
+    <value>true</value>
+  </property>
+
+  <property>
+    <name>fs.contract.rejects-seek-past-eof</name>
+    <value>true</value>
+  </property>
+
+  <property>
+    <name>fs.contract.supports-available-on-closed-file</name>
+    <value>true</value>
+  </property>
+
+  <property>
+    <name>fs.contract.supports-strict-exceptions</name>
+    <value>false</value>
+  </property>
+
+  <property>
+    <name>fs.contract.supports-unix-permissions</name>
+    <value>true</value>
+  </property>
+
+  <property>
+    <name>fs.contract.rename-overwrites-dest</name>
+    <value>true</value>
+  </property>
+
+  <property>
+    <name>fs.contract.supports-append</name>
+    <value>true</value>
+  </property>
+
+  <property>
+    <name>fs.azure.enable.append.support</name>
+    <value>true</value>
+  </property>
+
+  <property>
+    <name>fs.contract.supports-getfilestatus</name>
+    <value>true</value>
+  </property>
+</configuration>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c9e71382/hadoop-tools/hadoop-azure-datalake/src/test/resources/contract-test-options.xml
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure-datalake/src/test/resources/contract-test-options.xml b/hadoop-tools/hadoop-azure-datalake/src/test/resources/contract-test-options.xml
new file mode 100644
index 0000000..160fba4
--- /dev/null
+++ b/hadoop-tools/hadoop-azure-datalake/src/test/resources/contract-test-options.xml
@@ -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.
+-->
+<configuration>
+  <property>
+    <name>dfs.webhdfs.oauth2.refresh.token.expires.ms.since.epoch</name>
+    <value>0</value>
+  </property>
+  <property>
+    <name>dfs.webhdfs.oauth2.credential</name>
+    <value>bearer.and.refresh.token</value>
+  </property>
+  <property>
+    <name>dfs.webhdfs.oauth2.refresh.url</name>
+    <value>https://login.windows.net/common/oauth2/token/</value>
+  </property>
+  <property>
+    <name>dfs.webhdfs.oauth2.access.token.provider</name>
+    <value>
+      org.apache.hadoop.fs.adl.oauth2.CachedRefreshTokenBasedAccessTokenProvider
+    </value>
+  </property>
+  <property>
+    <name>dfs.webhdfs.oauth2.enabled</name>
+    <value>true</value>
+  </property>
+  <!--USER INPUT REQUIRED-->
+  <property>
+    <name>dfs.webhdfs.oauth2.client.id</name>
+    <value>ADD CLIENT ID</value>
+  </property>
+  <!--USER INPUT REQUIRED-->
+  <property>
+    <name>dfs.webhdfs.oauth2.refresh.token</name>
+    <value>ADD REFRESH TOKEN</value>
+  </property>
+  <!--USER INPUT REQUIRED-->
+  <property>
+    <name>fs.defaultFS</name>
+    <value>adl://urAdlAccountGoesHere.azuredatalakestore.net:443/</value>
+  </property>
+  <!--USER INPUT REQUIRED-->
+  <property>
+    <name>dfs.adl.test.contract.enable</name>
+    <value>false</value>
+  </property>
+</configuration>
\ No newline at end of file


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


[39/39] hadoop git commit: Merge branch 'trunk' into HDFS-1312

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


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

Branch: refs/heads/HDFS-1312
Commit: 91d62aaf188a90fc02eeb35c6180f4097a2d1246
Parents: 13599a8 0319d73
Author: Anu Engineer <ae...@apache.org>
Authored: Sat Jun 18 17:32:11 2016 -0700
Committer: Anu Engineer <ae...@apache.org>
Committed: Sat Jun 18 17:32:11 2016 -0700

----------------------------------------------------------------------
 .../crypto/key/kms/KMSClientProvider.java       |   2 -
 .../apache/hadoop/fs/CachingGetSpaceUsed.java   |  28 +-
 .../hadoop/fs/CommonConfigurationKeys.java      |   3 +
 .../src/main/java/org/apache/hadoop/fs/DU.java  |  10 +-
 .../java/org/apache/hadoop/fs/FileUtil.java     |  14 +
 .../java/org/apache/hadoop/fs/GetSpaceUsed.java |  25 ++
 .../apache/hadoop/fs/RawLocalFileSystem.java    |   5 -
 .../apache/hadoop/fs/WindowsGetSpaceUsed.java   |   9 +-
 .../java/org/apache/hadoop/fs/shell/Test.java   |  80 +++--
 .../org/apache/hadoop/ipc/CallQueueManager.java |   4 +-
 .../apache/hadoop/ipc/DecayRpcScheduler.java    |  17 +-
 .../org/apache/hadoop/ipc/FairCallQueue.java    |  19 +-
 .../hadoop/security/UserGroupInformation.java   |   2 +-
 .../web/DelegationTokenAuthenticator.java       |   3 +
 .../org/apache/hadoop/util/StringUtils.java     |  54 ++-
 .../src/main/resources/core-default.xml         |   1 +
 .../src/site/markdown/FileSystemShell.md        |   3 +
 .../test/java/org/apache/hadoop/fs/TestDU.java  |  10 +-
 .../org/apache/hadoop/fs/TestGetSpaceUsed.java  |   5 +-
 .../apache/hadoop/ipc/TestCallQueueManager.java |   4 +-
 .../hadoop/ipc/TestDecayRpcScheduler.java       |  30 +-
 .../apache/hadoop/ipc/TestFairCallQueue.java    |  27 +-
 .../hadoop/security/token/TestDtUtilShell.java  |   3 +-
 .../org/apache/hadoop/util/TestStringUtils.java |  44 ++-
 .../hadoop/crypto/key/kms/server/TestKMS.java   |  91 ++++-
 .../src/test/resources/log4j.properties         |   2 +-
 .../java/org/apache/hadoop/minikdc/MiniKdc.java |  11 +-
 .../hdfs/shortcircuit/TestShortCircuitShm.java  |   4 +-
 .../TestHttpFSFWithSWebhdfsFileSystem.java      |   5 +-
 .../TestHttpFSFileSystemLocalFileSystem.java    |   5 +-
 .../hdfs/nfs/nfs3/TestNfs3HttpServer.java       |   5 +-
 .../apache/hadoop/hdfs/HDFSPolicyProvider.java  |   6 +-
 .../BlockPlacementStatusWithUpgradeDomain.java  |   2 +-
 .../CacheReplicationMonitor.java                |   8 +
 .../server/namenode/FSImageFormatPBINode.java   |  10 +-
 .../OfflineImageReconstructor.java              |  30 +-
 .../src/main/webapps/hdfs/dfshealth.html        |   9 +
 .../src/main/webapps/hdfs/dfshealth.js          |  10 +
 .../src/site/markdown/HdfsRollingUpgrade.md     |   2 +-
 .../apache/hadoop/cli/TestCryptoAdminCLI.java   |   4 +-
 .../hadoop/fs/TestEnhancedByteBufferAccess.java |  12 +-
 .../TestSWebHdfsFileContextMainOperations.java  |   6 +-
 .../TestWebHdfsFileContextMainOperations.java   |   7 +-
 .../org/apache/hadoop/hdfs/MiniDFSCluster.java  |  17 +-
 .../hadoop/hdfs/TestAppendSnapshotTruncate.java |   4 +-
 .../org/apache/hadoop/hdfs/TestDFSShell.java    |  75 +++-
 .../hadoop/hdfs/TestDFSUpgradeFromImage.java    |  18 +-
 .../org/apache/hadoop/hdfs/TestDFSUtil.java     |   3 +-
 .../hadoop/hdfs/TestDatanodeLayoutUpgrade.java  |  21 +-
 ...estDatanodeStartupFixesLegacyStorageIDs.java |  12 +-
 .../org/apache/hadoop/hdfs/TestFetchImage.java  |   5 +-
 .../hadoop/hdfs/TestHDFSPolicyProvider.java     |  31 +-
 .../java/org/apache/hadoop/hdfs/TestPread.java  |   2 +-
 .../org/apache/hadoop/hdfs/TestSeekBug.java     |   3 +-
 .../sasl/SaslDataTransferTestCase.java          |   5 +-
 .../hdfs/qjournal/TestSecureNNWithQJM.java      |   5 +-
 .../hdfs/server/balancer/TestBalancer.java      |   3 +-
 ...stBlockPlacementStatusWithUpgradeDomain.java |  83 +++++
 .../hdfs/server/datanode/TestDataNodeUUID.java  |   3 +-
 .../hdfs/server/datanode/TestDataStorage.java   |   3 +-
 .../server/namenode/TestCreateEditsLog.java     |   6 +-
 .../namenode/TestDecommissioningStatus.java     |  65 +---
 .../namenode/TestEditLogFileInputStream.java    |   3 +-
 .../namenode/TestFSImageWithSnapshot.java       |   4 +-
 .../hadoop/hdfs/server/namenode/TestFsck.java   |  11 +-
 .../server/namenode/TestNameNodeHttpServer.java |   5 +-
 .../server/namenode/TestNameNodeRecovery.java   |   5 +-
 .../TestNameNodeRespectsBindHostKeys.java       |   5 +-
 .../TestUpgradeDomainBlockPlacementPolicy.java  | 161 +++++++--
 .../snapshot/TestRenameWithSnapshots.java       |   2 +-
 .../server/namenode/snapshot/TestSnapshot.java  |   4 +-
 .../apache/hadoop/hdfs/tools/TestGetConf.java   |  62 +---
 .../TestOfflineImageViewer.java                 |  19 ++
 .../hadoop/hdfs/util/HostsFileWriter.java       |  32 +-
 .../hdfs/util/TestCombinedHostsFileReader.java  |   5 +-
 .../hadoop/hdfs/web/TestHttpsFileSystem.java    |   5 +-
 .../hadoop/hdfs/web/TestWebHdfsTokens.java      |   9 +-
 .../java/org/apache/hadoop/test/PathUtils.java  |   7 +-
 .../HumanReadableHistoryViewerPrinter.java      |  15 +-
 .../hadoop-aws/dev-support/findbugs-exclude.xml |   7 +
 .../org/apache/hadoop/fs/s3/S3Credentials.java  |  31 +-
 .../org/apache/hadoop/fs/s3/S3FileSystem.java   |  20 +-
 .../org/apache/hadoop/fs/s3a/S3AFileSystem.java |  79 ++---
 .../java/org/apache/hadoop/fs/s3a/S3AUtils.java |  46 +++
 .../hadoop/fs/s3native/NativeS3FileSystem.java  |  19 +-
 .../hadoop/fs/s3native/S3xLoginHelper.java      | 283 +++++++++++++++
 .../src/site/markdown/tools/hadoop-aws/index.md | 313 ++++++++++++++++-
 .../apache/hadoop/fs/s3/TestS3FileSystem.java   |  24 +-
 .../hadoop/fs/s3a/TestS3AConfiguration.java     |  55 ++-
 .../hadoop/fs/s3a/TestS3ACredentialsInURL.java  | 153 +++++++++
 .../hadoop/fs/s3native/TestS3xLoginHelper.java  | 197 +++++++++++
 hadoop-tools/hadoop-azure-datalake/pom.xml      |   7 +-
 .../web/PrivateAzureDataLakeFileSystem.java     |   2 +-
 ...ClientCredentialBasedAccesTokenProvider.java | 155 +++++++++
 .../src/site/markdown/index.md                  |  64 ++++
 .../hadoop/fs/adl/TestADLResponseData.java      | 120 +++++++
 .../apache/hadoop/fs/adl/TestGetFileStatus.java |  65 ++++
 .../apache/hadoop/fs/adl/TestListStatus.java    |  95 ++++++
 .../hadoop/fs/adl/TestableAdlFileSystem.java    |  31 ++
 .../fs/adl/live/AdlStorageConfiguration.java    |  76 +++++
 .../hadoop/fs/adl/live/AdlStorageContract.java  |  65 ++++
 .../fs/adl/live/TestAdlContractAppendLive.java  |  53 +++
 .../fs/adl/live/TestAdlContractConcatLive.java  |  52 +++
 .../fs/adl/live/TestAdlContractCreateLive.java  |  52 +++
 .../fs/adl/live/TestAdlContractDeleteLive.java  |  44 +++
 .../fs/adl/live/TestAdlContractMkdirLive.java   |  55 +++
 .../fs/adl/live/TestAdlContractOpenLive.java    |  44 +++
 .../fs/adl/live/TestAdlContractRenameLive.java  |  63 ++++
 .../fs/adl/live/TestAdlContractRootDirLive.java |  52 +++
 .../fs/adl/live/TestAdlContractSeekLive.java    |  44 +++
 .../live/TestAdlDifferentSizeWritesLive.java    | 102 ++++++
 .../adl/live/TestAdlFileSystemContractLive.java | 111 ++++++
 .../hadoop/fs/adl/live/TestAdlReadLive.java     | 342 +++++++++++++++++++
 ...estAdlWebHdfsFileContextCreateMkdirLive.java |  79 +++++
 ...AdlWebHdfsFileContextMainOperationsLive.java | 104 ++++++
 ...hedRefreshTokenBasedAccessTokenProvider.java |   8 +-
 .../hadoop/fs/common/AdlMockWebServer.java      | 116 +++++++
 .../hadoop/fs/common/ExpectedResponse.java      |  72 ++++
 .../hadoop/fs/common/TestDataForRead.java       | 120 +++++++
 .../org/apache/hadoop/hdfs/web/TestAdlRead.java | 205 +++++++++++
 .../web/TestConcurrentDataReadOperations.java   | 306 +++++++++++++++++
 .../hdfs/web/TestConfigurationSetting.java      | 112 ++++++
 .../hdfs/web/TestSplitSizeCalculation.java      | 123 +++++++
 .../src/test/resources/adls.xml                 | 139 ++++++++
 .../test/resources/contract-test-options.xml    |  57 ++++
 .../apache/hadoop/yarn/client/cli/LogsCLI.java  |   7 +-
 .../hadoop/yarn/client/cli/TestLogsCLI.java     |  25 +-
 .../logaggregation/AggregatedLogFormat.java     |  74 ++--
 .../logaggregation/TestAggregatedLogFormat.java |   2 +-
 .../webapp/AHSWebServices.java                  |  11 +-
 .../logaggregation/AppLogAggregatorImpl.java    |   8 +-
 .../nodemanager/webapp/NMWebServices.java       |   9 +-
 .../TestLogAggregationService.java              | 102 +++++-
 .../scheduler/fair/FSLeafQueue.java             |  33 +-
 .../resourcemanager/scheduler/fair/FSQueue.java |  19 --
 .../scheduler/fair/SchedulingPolicy.java        |  11 -
 .../DominantResourceFairnessPolicy.java         |   5 -
 .../fair/policies/FairSharePolicy.java          |   5 -
 .../scheduler/fair/policies/FifoPolicy.java     |   5 -
 .../scheduler/fair/TestFairScheduler.java       |  82 ++++-
 140 files changed, 5481 insertions(+), 628 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/91d62aaf/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/balancer/TestBalancer.java
----------------------------------------------------------------------


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


[09/39] hadoop git commit: HADOOP-13245. Fix up some misc create-release issues (aw)

Posted by ae...@apache.org.
HADOOP-13245. Fix up some misc create-release issues (aw)


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

Branch: refs/heads/HDFS-1312
Commit: e2f640942b722e35490cf146c0268517da5a28b1
Parents: c77a109
Author: Allen Wittenauer <aw...@apache.org>
Authored: Tue Jun 14 16:03:20 2016 -0700
Committer: Allen Wittenauer <aw...@apache.org>
Committed: Tue Jun 14 16:03:20 2016 -0700

----------------------------------------------------------------------
 dev-support/bin/create-release              | 69 ++++++++++++++++++------
 dev-support/docker/Dockerfile               |  5 ++
 hadoop-common-project/hadoop-common/pom.xml |  3 +-
 pom.xml                                     |  2 +
 4 files changed, 62 insertions(+), 17 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/e2f64094/dev-support/bin/create-release
----------------------------------------------------------------------
diff --git a/dev-support/bin/create-release b/dev-support/bin/create-release
index 5ea47cd..0e0ab86 100755
--- a/dev-support/bin/create-release
+++ b/dev-support/bin/create-release
@@ -240,6 +240,8 @@ function set_defaults
   OSNAME=$(uname -s)
 
   PUBKEYFILE="https://dist.apache.org/repos/dist/release/hadoop/common/KEYS"
+
+  SIGN=false
 }
 
 function startgpgagent
@@ -247,11 +249,23 @@ function startgpgagent
   if [[ "${SIGN}" = true ]]; then
     if [[ -n "${GPGAGENT}" && -z "${GPG_AGENT_INFO}" ]]; then
       echo "starting gpg agent"
-      touch "${LOGDIR}/gpgagent.conf"
+      echo "default-cache-ttl 7200" > "${LOGDIR}/gpgagent.conf"
+      # shellcheck disable=2046
       eval $("${GPGAGENT}" --daemon \
         --options "${LOGDIR}/gpgagent.conf" \
-        --log-file=${LOGDIR}/create-release-gpgagent.log)
-      GPGAGENTPID=$(echo ${GPG_AGENT_INFO} | cut -f 2 -d:)
+        --log-file="${LOGDIR}/create-release-gpgagent.log")
+      GPGAGENTPID=$(echo "${GPG_AGENT_INFO}" | cut -f 2 -d:)
+    fi
+
+    if [[ -n "${GPG_AGENT_INFO}" ]]; then
+      echo "Warming the gpg-agent cache prior to calling maven"
+      # warm the agent's cache:
+      touch "${LOGDIR}/warm"
+      ${GPG} --use-agent --armor --output "${LOGDIR}/warm.asc" --detach-sig "${LOGDIR}/warm"
+      rm "${LOGDIR}/warm.asc" "${LOGDIR}/warm"
+    else
+      SIGN=false
+      hadoop_error "ERROR: Unable to launch or acquire gpg-agent. Disable signing."
     fi
   fi
 }
@@ -259,7 +273,7 @@ function startgpgagent
 function stopgpgagent
 {
   if [[ -n "${GPGAGENTPID}" ]]; then
-    kill ${GPGAGENTPID}
+    kill "${GPGAGENTPID}"
   fi
 }
 
@@ -273,7 +287,7 @@ function usage
   echo "--mvncache=[path]       Path to the maven cache to use"
   echo "--native                Also build the native components"
   echo "--rc-label=[label]      Add this label to the builds"
-  echo "--sign                  Use .gnupg dir to sign the jars"
+  echo "--sign                  Use .gnupg dir to sign the artifacts and jars"
   echo "--version=[version]     Use an alternative version string"
 }
 
@@ -330,6 +344,16 @@ function option_parse
     SIGN=false
   fi
 
+  if [[ "${SIGN}" = true ]]; then
+    if [[ -n "${GPG_AGENT_INFO}" ]]; then
+      echo "NOTE: Using existing gpg-agent. If the default-cache-ttl"
+      echo "is set to less than ~20 mins, maven commands will fail."
+    elif [[ -z "${GPGAGENT}" ]]; then
+      hadoop_error "ERROR: No gpg-agent. Disabling signing capability."
+      SIGN=false
+    fi
+  fi
+
   DOCKERCMD=$(command -v docker)
   if [[ "${DOCKER}" = true && -z "${DOCKERCMD}" ]]; then
       hadoop_error "ERROR: docker binary not found. Disabling docker mode."
@@ -439,6 +463,11 @@ function dockermode
     # make sure we put some space between, just in case last
     # line isn't an empty line or whatever
     printf "\n\n"
+
+    # force a new image for every run to make it easier to remove later
+    echo "LABEL org.apache.hadoop.create-release=\"cr-${RANDOM}\""
+
+    # setup ownerships, etc
     echo "RUN groupadd --non-unique -g ${group_id} ${user_name}"
     echo "RUN useradd -g ${group_id} -u ${user_id} -m ${user_name}"
     echo "RUN chown -R ${user_name} /home/${user_name}"
@@ -490,19 +519,27 @@ function makearelease
 
   big_console_header "Maven Build and Install"
 
+  if [[ "${SIGN}" = true ]]; then
+    signflags=("-Psign" "-Dgpg.useagent=true" -Dgpg.executable="${GPG}")
+  fi
+
   # Create SRC and BIN tarballs for release,
-  # Using 'install\u2019 goal instead of 'package' so artifacts are available
-  # in the Maven local cache for the site generation
-  #
   # shellcheck disable=SC2046
   run_and_redirect "${LOGDIR}/mvn_install.log" \
-    "${MVN}" "${MVN_ARGS[@]}" install -Pdist,src \
+    "${MVN}" "${MVN_ARGS[@]}" install \
+      -Pdist,src \
+      "${signflags[@]}" \
       -DskipTests -Dtar $(hadoop_native_flags)
 
-  big_console_header "Maven Site"
-
   # Create site for release
-  run_and_redirect "${LOGDIR}/mvn_site.log" "${MVN}" "${MVN_ARGS[@]}" site site:stage -Pdist,src,releasedocs
+  # we need to do install again so that jdiff and
+  # a few other things get registered in the maven
+  # universe correctly
+  run_and_redirect "${LOGDIR}/mvn_site.log" \
+    "${MVN}" "${MVN_ARGS[@]}" install \
+      site site:stage \
+      -DskipTests \
+      -Pdist,src,releasedocs,docs
 
   big_console_header "Staging the release"
 
@@ -560,16 +597,16 @@ function signartifacts
   big_console_header "Signing the release"
 
   for i in ${ARTIFACTS_DIR}/*; do
-    gpg --use-agent --armor --output "${i}.asc" --detach-sig "${i}"
-    gpg --print-mds "${i}" > "${i}.mds"
+    ${GPG} --use-agent --armor --output "${i}.asc" --detach-sig "${i}"
+    ${GPG} --print-mds "${i}" > "${i}.mds"
     domd5 "${i}"
   done
 
   if [[ "${ASFRELEASE}" = true ]]; then
     echo "Fetching the Apache Hadoop KEYS file..."
     curl -L "${PUBKEYFILE}" -o "${BASEDIR}/target/KEYS"
-    gpg --import --trustdb "${BASEDIR}/target/testkeysdb" "${BASEDIR}/target/KEYS"
-    gpg --verify --trustdb "${BASEDIR}/target/testkeysdb" \
+    ${GPG} --import --trustdb "${BASEDIR}/target/testkeysdb" "${BASEDIR}/target/KEYS"
+    ${GPG} --verify --trustdb "${BASEDIR}/target/testkeysdb" \
       "${ARTIFACTS_DIR}/hadoop-${HADOOP_VERSION}${RC_LABEL}.tar.gz.asc" \
         "${ARTIFACTS_DIR}/hadoop-${HADOOP_VERSION}${RC_LABEL}.tar.gz"
     if [[ $? != 0 ]]; then

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e2f64094/dev-support/docker/Dockerfile
----------------------------------------------------------------------
diff --git a/dev-support/docker/Dockerfile b/dev-support/docker/Dockerfile
index f9bf5aa..82edc86 100644
--- a/dev-support/docker/Dockerfile
+++ b/dev-support/docker/Dockerfile
@@ -129,6 +129,11 @@ RUN apt-get -q install --no-install-recommends -y bats
 ####
 RUN pip install pylint
 
+####
+# Install dateutil.parser
+####
+RUN pip install python-dateutil
+
 ###
 # Avoid out of memory errors in builds
 ###

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e2f64094/hadoop-common-project/hadoop-common/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/pom.xml b/hadoop-common-project/hadoop-common/pom.xml
index 059986f..51ff7de 100644
--- a/hadoop-common-project/hadoop-common/pom.xml
+++ b/hadoop-common-project/hadoop-common/pom.xml
@@ -985,11 +985,12 @@
                     </goals>
                     <configuration>
                         <executable>${basedir}/../../dev-support/bin/releasedocmaker</executable>
-                        <workingDirectory>src/site/markdown/release/</workingDirectory>
                         <requiresOnline>true</requiresOnline>
                         <arguments>
                             <argument>--index</argument>
                             <argument>--license</argument>
+                            <argument>--outputdir</argument>
+                            <argument>${basedir}/src/site/markdown/release</argument>
                             <argument>--project</argument>
                             <argument>HADOOP</argument>
                             <argument>--project</argument>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e2f64094/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index c908340..5bcf993 100644
--- a/pom.xml
+++ b/pom.xml
@@ -99,6 +99,7 @@ xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xs
     <maven-dependency-plugin.version>2.8</maven-dependency-plugin.version>
     <maven-enforcer-plugin.version>1.3.1</maven-enforcer-plugin.version>
     <maven-javadoc-plugin.version>2.9.1</maven-javadoc-plugin.version>
+    <maven-gpg-plugin.version>1.5</maven-gpg-plugin.version>
     <apache-rat-plugin.version>0.10</apache-rat-plugin.version>
     <wagon-ssh.version>1.0</wagon-ssh.version>
     <maven-clover2-plugin.version>3.3.0</maven-clover2-plugin.version>
@@ -586,6 +587,7 @@ xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xs
           <plugin>
             <groupId>org.apache.maven.plugins</groupId>
             <artifactId>maven-gpg-plugin</artifactId>
+            <version>${maven-gpg-plugin.version}</version>
             <executions>
               <execution>
                 <id>sign-artifacts</id>


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


[06/39] hadoop git commit: YARN-1942. Deprecate toString/fromString methods from ConverterUtils and move them to records classes like ContainerId/ApplicationId, etc. (wangda)

Posted by ae...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/c77a1095/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/TestContainerManager.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/TestContainerManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/TestContainerManager.java
index 10b9155..1f803b4 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/TestContainerManager.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/TestContainerManager.java
@@ -199,7 +199,7 @@ public class TestContainerManager extends BaseContainerManagerTest {
     ContainerLaunchContext containerLaunchContext = 
         recordFactory.newRecordInstance(ContainerLaunchContext.class);
     URL resource_alpha =
-        ConverterUtils.getYarnUrlFromPath(localFS
+        URL.fromPath(localFS
             .makeQualified(new Path(file.getAbsolutePath())));
     LocalResource rsrc_alpha = recordFactory.newRecordInstance(LocalResource.class);    
     rsrc_alpha.setResource(resource_alpha);
@@ -229,8 +229,8 @@ public class TestContainerManager extends BaseContainerManagerTest {
 
     // Now ascertain that the resources are localised correctly.
     ApplicationId appId = cId.getApplicationAttemptId().getApplicationId();
-    String appIDStr = ConverterUtils.toString(appId);
-    String containerIDStr = ConverterUtils.toString(cId);
+    String appIDStr = appId.toString();
+    String containerIDStr = cId.toString();
     File userCacheDir = new File(localDir, ContainerLocalizer.USERCACHE);
     File userDir = new File(userCacheDir, user);
     File appCache = new File(userDir, ContainerLocalizer.APPCACHE);
@@ -288,7 +288,7 @@ public class TestContainerManager extends BaseContainerManagerTest {
         recordFactory.newRecordInstance(ContainerLaunchContext.class);
 
     URL resource_alpha =
-        ConverterUtils.getYarnUrlFromPath(localFS
+        URL.fromPath(localFS
             .makeQualified(new Path(scriptFile.getAbsolutePath())));
     LocalResource rsrc_alpha =
         recordFactory.newRecordInstance(LocalResource.class);
@@ -395,7 +395,7 @@ public class TestContainerManager extends BaseContainerManagerTest {
 			  recordFactory.newRecordInstance(ContainerLaunchContext.class);
 
 	  URL resource_alpha =
-			  ConverterUtils.getYarnUrlFromPath(localFS
+			  URL.fromPath(localFS
 					  .makeQualified(new Path(scriptFile.getAbsolutePath())));
 	  LocalResource rsrc_alpha =
 			  recordFactory.newRecordInstance(LocalResource.class);
@@ -488,7 +488,7 @@ public class TestContainerManager extends BaseContainerManagerTest {
 //    containerLaunchContext.resources =
 //        new HashMap<CharSequence, LocalResource>();
     URL resource_alpha =
-        ConverterUtils.getYarnUrlFromPath(FileContext.getLocalFSFileContext()
+        URL.fromPath(FileContext.getLocalFSFileContext()
             .makeQualified(new Path(file.getAbsolutePath())));
     LocalResource rsrc_alpha = recordFactory.newRecordInstance(LocalResource.class);
     rsrc_alpha.setResource(resource_alpha);
@@ -521,8 +521,8 @@ public class TestContainerManager extends BaseContainerManagerTest {
         ApplicationState.RUNNING);
 
     // Now ascertain that the resources are localised correctly.
-    String appIDStr = ConverterUtils.toString(appId);
-    String containerIDStr = ConverterUtils.toString(cId);
+    String appIDStr = appId.toString();
+    String containerIDStr = cId.toString();
     File userCacheDir = new File(localDir, ContainerLocalizer.USERCACHE);
     File userDir = new File(userCacheDir, user);
     File appCache = new File(userDir, ContainerLocalizer.APPCACHE);
@@ -975,7 +975,7 @@ public class TestContainerManager extends BaseContainerManagerTest {
     ContainerLaunchContext containerLaunchContext =
         recordFactory.newRecordInstance(ContainerLaunchContext.class);
     URL resource_alpha =
-        ConverterUtils.getYarnUrlFromPath(localFS
+        URL.fromPath(localFS
             .makeQualified(new Path(scriptFile.getAbsolutePath())));
     LocalResource rsrc_alpha =
         recordFactory.newRecordInstance(LocalResource.class);
@@ -1059,7 +1059,7 @@ public class TestContainerManager extends BaseContainerManagerTest {
     ContainerLaunchContext containerLaunchContext =
         recordFactory.newRecordInstance(ContainerLaunchContext.class);
     URL resource_alpha =
-        ConverterUtils.getYarnUrlFromPath(localFS
+        URL.fromPath(localFS
             .makeQualified(new Path(scriptFile.getAbsolutePath())));
     LocalResource rsrc_alpha =
         recordFactory.newRecordInstance(LocalResource.class);
@@ -1181,7 +1181,7 @@ public class TestContainerManager extends BaseContainerManagerTest {
     ContainerId cId = createContainerId(0);
 
     URL resource_alpha =
-        ConverterUtils.getYarnUrlFromPath(localFS
+        URL.fromPath(localFS
             .makeQualified(new Path(scriptFile.getAbsolutePath())));
     LocalResource rsrc_alpha =
         recordFactory.newRecordInstance(LocalResource.class);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c77a1095/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/TestContainerManagerRecovery.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/TestContainerManagerRecovery.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/TestContainerManagerRecovery.java
index 61477a7..b7d0e48 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/TestContainerManagerRecovery.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/TestContainerManagerRecovery.java
@@ -415,7 +415,7 @@ public class TestContainerManagerRecovery extends BaseContainerManagerTest {
     fileWriter.close();
     FileContext localFS = FileContext.getLocalFSFileContext();
     URL resource_alpha =
-        ConverterUtils.getYarnUrlFromPath(localFS
+        URL.fromPath(localFS
             .makeQualified(new Path(scriptFile.getAbsolutePath())));
     LocalResource rsrc_alpha = RecordFactoryProvider
         .getRecordFactory(null).newRecordInstance(LocalResource.class);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c77a1095/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/launcher/TestContainerLaunch.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/launcher/TestContainerLaunch.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/launcher/TestContainerLaunch.java
index cf7ca8d..a558338 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/launcher/TestContainerLaunch.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/launcher/TestContainerLaunch.java
@@ -538,7 +538,7 @@ public class TestContainerLaunch extends BaseContainerManagerTest {
     when(container.getContainerId()).thenReturn(containerId);
     when(container.getUser()).thenReturn("test");
     String relativeContainerLogDir = ContainerLaunch.getRelativeContainerLogDir(
-        appId.toString(), ConverterUtils.toString(containerId));
+        appId.toString(), containerId.toString());
     Path containerLogDir =
         dirsHandler.getLogPathForWrite(relativeContainerLogDir, false);
 
@@ -744,7 +744,7 @@ public class TestContainerLaunch extends BaseContainerManagerTest {
 
     // upload the script file so that the container can run it
     URL resource_alpha =
-        ConverterUtils.getYarnUrlFromPath(localFS
+        URL.fromPath(localFS
             .makeQualified(new Path(scriptFile.getAbsolutePath())));
     LocalResource rsrc_alpha =
         recordFactory.newRecordInstance(LocalResource.class);
@@ -945,7 +945,7 @@ public class TestContainerLaunch extends BaseContainerManagerTest {
 
     // upload the script file so that the container can run it
     URL resource_alpha =
-        ConverterUtils.getYarnUrlFromPath(localFS
+        URL.fromPath(localFS
             .makeQualified(new Path(scriptFile.getAbsolutePath())));
     LocalResource rsrc_alpha =
         recordFactory.newRecordInstance(LocalResource.class);
@@ -1284,7 +1284,7 @@ public class TestContainerLaunch extends BaseContainerManagerTest {
 
     // upload the script file so that the container can run it
     URL resource_alpha =
-        ConverterUtils.getYarnUrlFromPath(localFS
+        URL.fromPath(localFS
             .makeQualified(new Path(scriptFile.getAbsolutePath())));
     LocalResource rsrc_alpha =
         recordFactory.newRecordInstance(LocalResource.class);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c77a1095/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/TestContainerLocalizer.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/TestContainerLocalizer.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/TestContainerLocalizer.java
index 611fc05..fac7086 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/TestContainerLocalizer.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/TestContainerLocalizer.java
@@ -404,7 +404,7 @@ public class TestContainerLocalizer {
 
     when(resourceLocalizationSpec.getResource()).thenReturn(rsrc);
     when(resourceLocalizationSpec.getDestinationDirectory()).
-      thenReturn(ConverterUtils.getYarnUrlFromPath(p));
+      thenReturn(URL.fromPath(p));
     return resourceLocalizationSpec;
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c77a1095/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/TestLocalResource.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/TestLocalResource.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/TestLocalResource.java
index 81446f5..13310ad 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/TestLocalResource.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/TestLocalResource.java
@@ -24,6 +24,7 @@ import java.util.Random;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.yarn.api.records.LocalResourceType;
 import org.apache.hadoop.yarn.api.records.LocalResourceVisibility;
+import org.apache.hadoop.yarn.api.records.URL;
 import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.LocalResourceRequest;
 import org.apache.hadoop.yarn.util.ConverterUtils;
@@ -39,8 +40,10 @@ public class TestLocalResource {
   static org.apache.hadoop.yarn.api.records.LocalResource getYarnResource(Path p, long size,
       long timestamp, LocalResourceType type, LocalResourceVisibility state, String pattern)
       throws URISyntaxException {
-    org.apache.hadoop.yarn.api.records.LocalResource ret = RecordFactoryProvider.getRecordFactory(null).newRecordInstance(org.apache.hadoop.yarn.api.records.LocalResource.class);
-    ret.setResource(ConverterUtils.getYarnUrlFromURI(p.toUri()));
+    org.apache.hadoop.yarn.api.records.LocalResource ret =
+        RecordFactoryProvider.getRecordFactory(null).newRecordInstance(
+            org.apache.hadoop.yarn.api.records.LocalResource.class);
+    ret.setResource(URL.fromURI(p.toUri()));
     ret.setSize(size);
     ret.setTimestamp(timestamp);
     ret.setType(type);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c77a1095/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/TestResourceLocalizationService.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/TestResourceLocalizationService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/TestResourceLocalizationService.java
index c612c14..f594d8c 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/TestResourceLocalizationService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/TestResourceLocalizationService.java
@@ -945,7 +945,7 @@ public class TestResourceLocalizationService {
       // Sigh. Thread init of private localizer not accessible
       Thread.sleep(1000);
       dispatcher.await();
-      String appStr = ConverterUtils.toString(appId);
+      String appStr = appId.toString();
       String ctnrStr = c.getContainerId().toString();
       ArgumentCaptor<LocalizerStartContext> contextCaptor = ArgumentCaptor
           .forClass(LocalizerStartContext.class);
@@ -2144,12 +2144,16 @@ public class TestResourceLocalizationService {
       // removing pending download request.
       spyService.getPublicLocalizer().pending.clear();
 
+      LocalizerContext lc = mock(LocalizerContext.class);
+      when(lc.getContainerId()).thenReturn(ContainerId.newContainerId(
+          ApplicationAttemptId.newInstance(ApplicationId.newInstance(1L, 1), 1),
+          1L));
+
       // Now I need to simulate a race condition wherein Event is added to
       // dispatcher before resource state changes to either FAILED or LOCALIZED
       // Hence sending event directly to dispatcher.
       LocalizerResourceRequestEvent localizerEvent =
-          new LocalizerResourceRequestEvent(lr, null,
-            mock(LocalizerContext.class), null);
+          new LocalizerResourceRequestEvent(lr, null, lc, null);
 
       dispatcher1.getEventHandler().handle(localizerEvent);
       // Waiting for download to start. This should return false as new download
@@ -2457,7 +2461,7 @@ public class TestResourceLocalizationService {
         BuilderUtils.newApplicationId(314159265358979L, 3);
     when(app.getUser()).thenReturn(user);
     when(app.getAppId()).thenReturn(appId);
-    when(app.toString()).thenReturn(ConverterUtils.toString(appId));
+    when(app.toString()).thenReturn(appId.toString());
 
     // init container.
     final Container c = getMockContainer(appId, 42, user);
@@ -2468,17 +2472,16 @@ public class TestResourceLocalizationService {
       Path usersdir = new Path(tmpDirs.get(i), ContainerLocalizer.USERCACHE);
       Path userdir = new Path(usersdir, user);
       Path allAppsdir = new Path(userdir, ContainerLocalizer.APPCACHE);
-      Path appDir = new Path(allAppsdir, ConverterUtils.toString(appId));
+      Path appDir = new Path(allAppsdir, appId.toString());
       Path containerDir =
-          new Path(appDir, ConverterUtils.toString(c.getContainerId()));
+          new Path(appDir, c.getContainerId().toString());
       containerLocalDirs.add(containerDir);
       appLocalDirs.add(appDir);
 
       Path sysDir =
           new Path(tmpDirs.get(i), ResourceLocalizationService.NM_PRIVATE_DIR);
-      Path appSysDir = new Path(sysDir, ConverterUtils.toString(appId));
-      Path containerSysDir =
-          new Path(appSysDir, ConverterUtils.toString(c.getContainerId()));
+      Path appSysDir = new Path(sysDir, appId.toString());
+      Path containerSysDir = new Path(appSysDir, c.getContainerId().toString());
 
       nmLocalContainerDirs.add(containerSysDir);
       nmLocalAppDirs.add(appSysDir);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c77a1095/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/logaggregation/TestAppLogAggregatorImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/logaggregation/TestAppLogAggregatorImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/logaggregation/TestAppLogAggregatorImpl.java
index 0127923..f929ca8 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/logaggregation/TestAppLogAggregatorImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/logaggregation/TestAppLogAggregatorImpl.java
@@ -108,10 +108,8 @@ public class TestAppLogAggregatorImpl {
     final ContainerId containerId = ContainerId.newContainerId(attemptId, 0);
 
     // create artificial log files
-    final File appLogDir = new File(LOCAL_LOG_DIR,
-        ConverterUtils.toString(applicationId));
-    final File containerLogDir = new File(appLogDir,
-        ConverterUtils.toString(containerId));
+    final File appLogDir = new File(LOCAL_LOG_DIR, applicationId.toString());
+    final File containerLogDir = new File(appLogDir, containerId.toString());
     containerLogDir.mkdirs();
     final Set<File> logFiles = createContainerLogFiles(containerLogDir, 3);
 
@@ -135,9 +133,9 @@ public class TestAppLogAggregatorImpl {
 
     // create artificial log files
     final File appLogDir = new File(LOCAL_LOG_DIR,
-        ConverterUtils.toString(applicationId));
+        applicationId.toString());
     final File containerLogDir = new File(appLogDir,
-        ConverterUtils.toString(containerId));
+        containerId.toString());
     containerLogDir.mkdirs();
     final Set<File> logFiles = createContainerLogFiles(containerLogDir, 3);
 
@@ -163,9 +161,9 @@ public class TestAppLogAggregatorImpl {
 
     // create artificial log files
     final File appLogDir = new File(LOCAL_LOG_DIR,
-        ConverterUtils.toString(applicationId));
+        applicationId.toString());
     final File containerLogDir = new File(appLogDir,
-        ConverterUtils.toString(containerId));
+        containerId.toString());
     containerLogDir.mkdirs();
     final Set<File> logFiles = createContainerLogFiles(containerLogDir, 3);
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c77a1095/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/logaggregation/TestLogAggregationService.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/logaggregation/TestLogAggregationService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/logaggregation/TestLogAggregationService.java
index 3961e1a..92c6b80 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/logaggregation/TestLogAggregationService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/logaggregation/TestLogAggregationService.java
@@ -194,7 +194,7 @@ public class TestLogAggregationService extends BaseContainerManagerTest {
 
     // AppLogDir should be created
     File app1LogDir =
-        new File(localLogDir, ConverterUtils.toString(application1));
+        new File(localLogDir, application1.toString());
     app1LogDir.mkdir();
     logAggregationService
         .handle(new LogHandlerAppStartedEvent(
@@ -221,7 +221,7 @@ public class TestLogAggregationService extends BaseContainerManagerTest {
     verify(delSrvc).delete(eq(user), eq((Path) null),
       eq(new Path(app1LogDir.getAbsolutePath())));
     
-    String containerIdStr = ConverterUtils.toString(container11);
+    String containerIdStr = container11.toString();
     File containerLogDir = new File(app1LogDir, containerIdStr);
     int count = 0;
     int maxAttempts = 50;
@@ -315,7 +315,7 @@ public class TestLogAggregationService extends BaseContainerManagerTest {
     logAggregationService.start();
 
     ApplicationId app = BuilderUtils.newApplicationId(1234, 1);
-    File appLogDir = new File(localLogDir, ConverterUtils.toString(app));
+    File appLogDir = new File(localLogDir, app.toString());
     appLogDir.mkdir();
     LogAggregationContext context =
         LogAggregationContext.newInstance("HOST*", "sys*");
@@ -352,7 +352,7 @@ public class TestLogAggregationService extends BaseContainerManagerTest {
 
     // AppLogDir should be created
     File app1LogDir =
-      new File(localLogDir, ConverterUtils.toString(application1));
+      new File(localLogDir, application1.toString());
     app1LogDir.mkdir();
     logAggregationService
         .handle(new LogHandlerAppStartedEvent(
@@ -402,7 +402,7 @@ public class TestLogAggregationService extends BaseContainerManagerTest {
 
     // AppLogDir should be created
     File app1LogDir =
-      new File(localLogDir, ConverterUtils.toString(application1));
+      new File(localLogDir, application1.toString());
     app1LogDir.mkdir();
     logAggregationService
         .handle(new LogHandlerAppStartedEvent(
@@ -423,7 +423,7 @@ public class TestLogAggregationService extends BaseContainerManagerTest {
         BuilderUtils.newApplicationAttemptId(application2, 1);
 
     File app2LogDir =
-      new File(localLogDir, ConverterUtils.toString(application2));
+      new File(localLogDir, application2.toString());
     app2LogDir.mkdir();
     LogAggregationContext contextWithAMOnly =
         Records.newRecord(LogAggregationContext.class);
@@ -452,7 +452,7 @@ public class TestLogAggregationService extends BaseContainerManagerTest {
         BuilderUtils.newApplicationAttemptId(application3, 1);
 
     File app3LogDir =
-      new File(localLogDir, ConverterUtils.toString(application3));
+      new File(localLogDir, application3.toString());
     app3LogDir.mkdir();
     LogAggregationContext contextWithAMAndFailed =
         Records.newRecord(LogAggregationContext.class);
@@ -583,7 +583,7 @@ public class TestLogAggregationService extends BaseContainerManagerTest {
         BuilderUtils.newApplicationId(System.currentTimeMillis(),
           (int) (Math.random() * 1000));
     File appLogDir =
-        new File(localLogDir, ConverterUtils.toString(appId2));
+        new File(localLogDir, appId2.toString());
     appLogDir.mkdir();
     logAggregationService.handle(new LogHandlerAppStartedEvent(appId2,
         this.user, null, this.acls, contextWithAMAndFailed));
@@ -758,7 +758,7 @@ public class TestLogAggregationService extends BaseContainerManagerTest {
           (int) (Math.random() * 1000));
 
     File appLogDir =
-        new File(localLogDir, ConverterUtils.toString(appId));
+        new File(localLogDir, appId.toString());
     appLogDir.mkdir();
 
     Exception e = new RuntimeException("KABOOM!");
@@ -805,7 +805,7 @@ public class TestLogAggregationService extends BaseContainerManagerTest {
   private void writeContainerLogs(File appLogDir, ContainerId containerId,
       String[] fileName) throws IOException {
     // ContainerLogDir should be created
-    String containerStr = ConverterUtils.toString(containerId);
+    String containerStr = containerId.toString();
     File containerLogDir = new File(appLogDir, containerStr);
     boolean created = containerLogDir.mkdirs();
     LOG.info("Created Dir:" + containerLogDir.getAbsolutePath() + " status :"
@@ -943,7 +943,7 @@ public class TestLogAggregationService extends BaseContainerManagerTest {
       Assert.assertTrue("number of containers with logs should be at most " +
           minNumOfContainers,logMap.size() <= maxNumOfContainers);
       for (ContainerId cId : expectedContainerIds) {
-        String containerStr = ConverterUtils.toString(cId);
+        String containerStr = cId.toString();
         Map<String, String> thisContainerMap = logMap.remove(containerStr);
         Assert.assertEquals(numOfLogsPerContainer, thisContainerMap.size());
         for (String fileType : logFiles) {
@@ -998,7 +998,7 @@ public class TestLogAggregationService extends BaseContainerManagerTest {
     ContainerId cId = BuilderUtils.newContainerId(appAttemptId, 0);
 
     URL resource_alpha =
-        ConverterUtils.getYarnUrlFromPath(localFS
+        URL.fromPath(localFS
             .makeQualified(new Path(scriptFile.getAbsolutePath())));
     LocalResource rsrc_alpha =
         recordFactory.newRecordInstance(LocalResource.class);
@@ -1435,7 +1435,7 @@ public class TestLogAggregationService extends BaseContainerManagerTest {
     // has only logs from stdout and syslog
     // AppLogDir should be created
     File appLogDir1 =
-        new File(localLogDir, ConverterUtils.toString(application1));
+        new File(localLogDir, application1.toString());
     appLogDir1.mkdir();
     logAggregationService.handle(new LogHandlerAppStartedEvent(application1,
       this.user, null, this.acls,
@@ -1460,7 +1460,7 @@ public class TestLogAggregationService extends BaseContainerManagerTest {
         BuilderUtils.newApplicationAttemptId(application2, 1);
 
     File app2LogDir =
-        new File(localLogDir, ConverterUtils.toString(application2));
+        new File(localLogDir, application2.toString());
     app2LogDir.mkdir();
     LogAggregationContextWithExcludePatterns.setLogAggregationPolicyClassName(
         AMOnlyLogAggregationPolicy.class.getName());
@@ -1485,7 +1485,7 @@ public class TestLogAggregationService extends BaseContainerManagerTest {
     ApplicationAttemptId appAttemptId3 =
         BuilderUtils.newApplicationAttemptId(application3, 1);
     File app3LogDir =
-        new File(localLogDir, ConverterUtils.toString(application3));
+        new File(localLogDir, application3.toString());
     app3LogDir.mkdir();
     context1.setLogAggregationPolicyClassName(
         AMOnlyLogAggregationPolicy.class.getName());
@@ -1510,7 +1510,7 @@ public class TestLogAggregationService extends BaseContainerManagerTest {
     ApplicationAttemptId appAttemptId4 =
         BuilderUtils.newApplicationAttemptId(application4, 1);
     File app4LogDir =
-        new File(localLogDir, ConverterUtils.toString(application4));
+        new File(localLogDir, application4.toString());
     app4LogDir.mkdir();
     context2.setLogAggregationPolicyClassName(
         AMOnlyLogAggregationPolicy.class.getName());
@@ -2012,7 +2012,7 @@ public class TestLogAggregationService extends BaseContainerManagerTest {
         containerType);
     // Simulate log-file creation
     File appLogDir1 =
-        new File(localLogDir, ConverterUtils.toString(application1));
+        new File(localLogDir, application1.toString());
     appLogDir1.mkdir();
     writeContainerLogs(appLogDir1, containerId, logFiles);
 
@@ -2123,7 +2123,7 @@ public class TestLogAggregationService extends BaseContainerManagerTest {
 
     // AppLogDir should be created
     File appLogDir =
-        new File(localLogDir, ConverterUtils.toString(application));
+        new File(localLogDir, application.toString());
     appLogDir.mkdir();
     logAggregationService.handle(new LogHandlerAppStartedEvent(application,
       this.user, null, this.acls, logAggregationContextWithInterval));

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c77a1095/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/TestContainersMonitor.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/TestContainersMonitor.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/TestContainersMonitor.java
index 94145e4..1b4e3b7 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/TestContainersMonitor.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/TestContainersMonitor.java
@@ -210,7 +210,7 @@ public class TestContainersMonitor extends BaseContainerManagerTest {
     ContainerId cId = ContainerId.newContainerId(appAttemptId, 0);
 
     URL resource_alpha =
-        ConverterUtils.getYarnUrlFromPath(localFS
+        URL.fromPath(localFS
             .makeQualified(new Path(scriptFile.getAbsolutePath())));
     LocalResource rsrc_alpha =
         recordFactory.newRecordInstance(LocalResource.class);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c77a1095/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/recovery/TestNMLeveldbStateStoreService.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/recovery/TestNMLeveldbStateStoreService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/recovery/TestNMLeveldbStateStoreService.java
index 2f409c8..d254e4b 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/recovery/TestNMLeveldbStateStoreService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/recovery/TestNMLeveldbStateStoreService.java
@@ -374,7 +374,7 @@ public class TestNMLeveldbStateStoreService {
     Path appRsrcPath = new Path("hdfs://some/app/resource");
     LocalResourcePBImpl rsrcPb = (LocalResourcePBImpl)
         LocalResource.newInstance(
-            ConverterUtils.getYarnUrlFromPath(appRsrcPath),
+            URL.fromPath(appRsrcPath),
             LocalResourceType.ARCHIVE, LocalResourceVisibility.APPLICATION,
             123L, 456L);
     LocalResourceProto appRsrcProto = rsrcPb.getProto();
@@ -407,7 +407,7 @@ public class TestNMLeveldbStateStoreService {
     // start some public and private resources
     Path pubRsrcPath1 = new Path("hdfs://some/public/resource1");
     rsrcPb = (LocalResourcePBImpl) LocalResource.newInstance(
-            ConverterUtils.getYarnUrlFromPath(pubRsrcPath1),
+            URL.fromPath(pubRsrcPath1),
             LocalResourceType.FILE, LocalResourceVisibility.PUBLIC,
             789L, 135L);
     LocalResourceProto pubRsrcProto1 = rsrcPb.getProto();
@@ -416,7 +416,7 @@ public class TestNMLeveldbStateStoreService {
         pubRsrcLocalPath1);
     Path pubRsrcPath2 = new Path("hdfs://some/public/resource2");
     rsrcPb = (LocalResourcePBImpl) LocalResource.newInstance(
-            ConverterUtils.getYarnUrlFromPath(pubRsrcPath2),
+            URL.fromPath(pubRsrcPath2),
             LocalResourceType.FILE, LocalResourceVisibility.PUBLIC,
             789L, 135L);
     LocalResourceProto pubRsrcProto2 = rsrcPb.getProto();
@@ -425,7 +425,7 @@ public class TestNMLeveldbStateStoreService {
         pubRsrcLocalPath2);
     Path privRsrcPath = new Path("hdfs://some/private/resource");
     rsrcPb = (LocalResourcePBImpl) LocalResource.newInstance(
-            ConverterUtils.getYarnUrlFromPath(privRsrcPath),
+            URL.fromPath(privRsrcPath),
             LocalResourceType.PATTERN, LocalResourceVisibility.PRIVATE,
             789L, 680L, "*pattern*");
     LocalResourceProto privRsrcProto = rsrcPb.getProto();
@@ -470,7 +470,7 @@ public class TestNMLeveldbStateStoreService {
     Path appRsrcPath = new Path("hdfs://some/app/resource");
     LocalResourcePBImpl rsrcPb = (LocalResourcePBImpl)
         LocalResource.newInstance(
-            ConverterUtils.getYarnUrlFromPath(appRsrcPath),
+            URL.fromPath(appRsrcPath),
             LocalResourceType.ARCHIVE, LocalResourceVisibility.APPLICATION,
             123L, 456L);
     LocalResourceProto appRsrcProto = rsrcPb.getProto();
@@ -510,7 +510,7 @@ public class TestNMLeveldbStateStoreService {
     // start some public and private resources
     Path pubRsrcPath1 = new Path("hdfs://some/public/resource1");
     rsrcPb = (LocalResourcePBImpl) LocalResource.newInstance(
-            ConverterUtils.getYarnUrlFromPath(pubRsrcPath1),
+            URL.fromPath(pubRsrcPath1),
             LocalResourceType.FILE, LocalResourceVisibility.PUBLIC,
             789L, 135L);
     LocalResourceProto pubRsrcProto1 = rsrcPb.getProto();
@@ -519,7 +519,7 @@ public class TestNMLeveldbStateStoreService {
         pubRsrcLocalPath1);
     Path pubRsrcPath2 = new Path("hdfs://some/public/resource2");
     rsrcPb = (LocalResourcePBImpl) LocalResource.newInstance(
-            ConverterUtils.getYarnUrlFromPath(pubRsrcPath2),
+            URL.fromPath(pubRsrcPath2),
             LocalResourceType.FILE, LocalResourceVisibility.PUBLIC,
             789L, 135L);
     LocalResourceProto pubRsrcProto2 = rsrcPb.getProto();
@@ -528,7 +528,7 @@ public class TestNMLeveldbStateStoreService {
         pubRsrcLocalPath2);
     Path privRsrcPath = new Path("hdfs://some/private/resource");
     rsrcPb = (LocalResourcePBImpl) LocalResource.newInstance(
-            ConverterUtils.getYarnUrlFromPath(privRsrcPath),
+            URL.fromPath(privRsrcPath),
             LocalResourceType.PATTERN, LocalResourceVisibility.PRIVATE,
             789L, 680L, "*pattern*");
     LocalResourceProto privRsrcProto = rsrcPb.getProto();
@@ -589,7 +589,7 @@ public class TestNMLeveldbStateStoreService {
     Path appRsrcPath = new Path("hdfs://some/app/resource");
     LocalResourcePBImpl rsrcPb = (LocalResourcePBImpl)
         LocalResource.newInstance(
-            ConverterUtils.getYarnUrlFromPath(appRsrcPath),
+            URL.fromPath(appRsrcPath),
             LocalResourceType.ARCHIVE, LocalResourceVisibility.APPLICATION,
             123L, 456L);
     LocalResourceProto appRsrcProto = rsrcPb.getProto();
@@ -619,7 +619,7 @@ public class TestNMLeveldbStateStoreService {
     // add public and private resources and remove some
     Path pubRsrcPath1 = new Path("hdfs://some/public/resource1");
     rsrcPb = (LocalResourcePBImpl) LocalResource.newInstance(
-            ConverterUtils.getYarnUrlFromPath(pubRsrcPath1),
+            URL.fromPath(pubRsrcPath1),
             LocalResourceType.FILE, LocalResourceVisibility.PUBLIC,
             789L, 135L);
     LocalResourceProto pubRsrcProto1 = rsrcPb.getProto();
@@ -635,7 +635,7 @@ public class TestNMLeveldbStateStoreService {
     stateStore.finishResourceLocalization(null, null, pubLocalizedProto1);
     Path pubRsrcPath2 = new Path("hdfs://some/public/resource2");
     rsrcPb = (LocalResourcePBImpl) LocalResource.newInstance(
-            ConverterUtils.getYarnUrlFromPath(pubRsrcPath2),
+            URL.fromPath(pubRsrcPath2),
             LocalResourceType.FILE, LocalResourceVisibility.PUBLIC,
             789L, 135L);
     LocalResourceProto pubRsrcProto2 = rsrcPb.getProto();
@@ -652,7 +652,7 @@ public class TestNMLeveldbStateStoreService {
     stateStore.removeLocalizedResource(null, null, pubRsrcLocalPath2);
     Path privRsrcPath = new Path("hdfs://some/private/resource");
     rsrcPb = (LocalResourcePBImpl) LocalResource.newInstance(
-            ConverterUtils.getYarnUrlFromPath(privRsrcPath),
+            URL.fromPath(privRsrcPath),
             LocalResourceType.PATTERN, LocalResourceVisibility.PRIVATE,
             789L, 680L, "*pattern*");
     LocalResourceProto privRsrcProto = rsrcPb.getProto();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c77a1095/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/webapp/TestNMWebServer.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/webapp/TestNMWebServer.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/webapp/TestNMWebServer.java
index b90c1be..3f71179 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/webapp/TestNMWebServer.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/webapp/TestNMWebServer.java
@@ -249,7 +249,7 @@ public class TestNMWebServer {
     containerLogDir.mkdirs();
     for (String fileType : new String[] { "stdout", "stderr", "syslog" }) {
       Writer writer = new FileWriter(new File(containerLogDir, fileType));
-      writer.write(ConverterUtils.toString(containerId) + "\n Hello "
+      writer.write(containerId.toString() + "\n Hello "
           + fileType + "!");
       writer.close();
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c77a1095/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/webapp/TestNMWebServicesContainers.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/webapp/TestNMWebServicesContainers.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/webapp/TestNMWebServicesContainers.java
index 18239f1..7ec8f27 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/webapp/TestNMWebServicesContainers.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/webapp/TestNMWebServicesContainers.java
@@ -40,6 +40,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.util.NodeHealthScriptRunner;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
+import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.event.AsyncDispatcher;
@@ -280,7 +281,7 @@ public class TestNMWebServicesContainers extends JerseyTestBase {
       verifyNodeContainerInfo(
           conInfo.getJSONObject(i),
           nmContext.getContainers().get(
-              ConverterUtils.toContainerId(conInfo.getJSONObject(i).getString(
+              ContainerId.fromString(conInfo.getJSONObject(i).getString(
                   "id"))));
     }
   }
@@ -316,7 +317,7 @@ public class TestNMWebServicesContainers extends JerseyTestBase {
       assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
       JSONObject json = response.getEntity(JSONObject.class);
       verifyNodeContainerInfo(json.getJSONObject("container"), nmContext
-          .getContainers().get(ConverterUtils.toContainerId(id)));
+          .getContainers().get(ContainerId.fromString(id)));
     }
   }
 
@@ -449,7 +450,7 @@ public class TestNMWebServicesContainers extends JerseyTestBase {
       NodeList nodes = dom.getElementsByTagName("container");
       assertEquals("incorrect number of elements", 1, nodes.getLength());
       verifyContainersInfoXML(nodes,
-          nmContext.getContainers().get(ConverterUtils.toContainerId(id)));
+          nmContext.getContainers().get(ContainerId.fromString(id)));
 
     }
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c77a1095/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 3634107..e36d96b 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
@@ -1345,7 +1345,7 @@ public class ResourceManager extends CompositeService implements Recoverable {
     rmStore.init(conf);
     rmStore.start();
     try {
-      ApplicationId removeAppId = ConverterUtils.toApplicationId(applicationId);
+      ApplicationId removeAppId = ApplicationId.fromString(applicationId);
       LOG.info("Deleting application " + removeAppId + " from state store");
       rmStore.removeApplication(removeAppId);
       LOG.info("Application is deleted from state store");

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c77a1095/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/LeveldbRMStateStore.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/LeveldbRMStateStore.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/LeveldbRMStateStore.java
index a6f0969..02f90dd 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/LeveldbRMStateStore.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/LeveldbRMStateStore.java
@@ -499,7 +499,7 @@ public class LeveldbRMStateStore extends RMStateStore {
 
   private ApplicationStateData createApplicationState(String appIdStr,
       byte[] data) throws IOException {
-    ApplicationId appId = ConverterUtils.toApplicationId(appIdStr);
+    ApplicationId appId = ApplicationId.fromString(appIdStr);
     ApplicationStateDataPBImpl appState =
         new ApplicationStateDataPBImpl(
             ApplicationStateDataProto.parseFrom(data));
@@ -545,8 +545,7 @@ public class LeveldbRMStateStore extends RMStateStore {
 
   private ApplicationAttemptStateData createAttemptState(String itemName,
       byte[] data) throws IOException {
-    ApplicationAttemptId attemptId =
-        ConverterUtils.toApplicationAttemptId(itemName);
+    ApplicationAttemptId attemptId = ApplicationAttemptId.fromString(itemName);
     ApplicationAttemptStateDataPBImpl attemptState =
         new ApplicationAttemptStateDataPBImpl(
             ApplicationAttemptStateDataProto.parseFrom(data));

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c77a1095/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/ZKRMStateStore.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/ZKRMStateStore.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/ZKRMStateStore.java
index 9afbf6d..9e05f6d 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/ZKRMStateStore.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/ZKRMStateStore.java
@@ -514,7 +514,7 @@ public class ZKRMStateStore extends RMStateStore {
         if (LOG.isDebugEnabled()) {
           LOG.debug("Loading application from znode: " + childNodeName);
         }
-        ApplicationId appId = ConverterUtils.toApplicationId(childNodeName);
+        ApplicationId appId = ApplicationId.fromString(childNodeName);
         ApplicationStateDataPBImpl appState =
             new ApplicationStateDataPBImpl(
                 ApplicationStateDataProto.parseFrom(childData));

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c77a1095/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/resource/DynamicResourceConfiguration.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/resource/DynamicResourceConfiguration.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/resource/DynamicResourceConfiguration.java
index 045c7bd..6549102 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/resource/DynamicResourceConfiguration.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/resource/DynamicResourceConfiguration.java
@@ -133,7 +133,7 @@ public class DynamicResourceConfiguration extends Configuration {
       = new HashMap<NodeId, ResourceOption> ();
 
     for (String node : nodes) {
-      NodeId nid = ConverterUtils.toNodeId(node);
+      NodeId nid = NodeId.fromString(node);
       int vcores = getVcoresPerNode(node);
       int memory = getMemoryPerNode(node);
       int overCommitTimeout = getOverCommitTimeoutPerNode(node);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c77a1095/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainerImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainerImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainerImpl.java
index 5121493..95f81d4 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainerImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainerImpl.java
@@ -346,7 +346,7 @@ public class RMContainerImpl implements RMContainer, Comparable<RMContainer> {
       logURL.append(WebAppUtils.getHttpSchemePrefix(rmContext
           .getYarnConfiguration()));
       logURL.append(WebAppUtils.getRunningLogURL(
-          container.getNodeHttpAddress(), ConverterUtils.toString(containerId),
+          container.getNodeHttpAddress(), containerId.toString(),
           user));
       return logURL.toString();
     } finally {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c77a1095/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMAppsBlock.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/RMAppsBlock.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMAppsBlock.java
index b4d7921..305f1d5 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMAppsBlock.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMAppsBlock.java
@@ -83,8 +83,8 @@ public class RMAppsBlock extends AppsBlock {
       }
 
       AppInfo app = new AppInfo(appReport);
-      ApplicationAttemptId appAttemptId =
-          ConverterUtils.toApplicationAttemptId(app.getCurrentAppAttemptId());
+      ApplicationAttemptId appAttemptId = ApplicationAttemptId.fromString(
+          app.getCurrentAppAttemptId());
       String queuePercent = "N/A";
       String clusterPercent = "N/A";
       if(appReport.getApplicationResourceUsageReport() != null) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c77a1095/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMWebAppFilter.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/RMWebAppFilter.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMWebAppFilter.java
index de2a23f..0f1a590 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMWebAppFilter.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMWebAppFilter.java
@@ -220,7 +220,7 @@ public class RMWebAppFilter extends GuiceContainer {
           break;
         case "appattempt":
           try{
-            appAttemptId = ConverterUtils.toApplicationAttemptId(parts[3]);
+            appAttemptId = ApplicationAttemptId.fromString(parts[3]);
           } catch (IllegalArgumentException e) {
             LOG.debug("Error parsing {} as an ApplicationAttemptId",
                 parts[3], e);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c77a1095/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 d05d952..878bf65 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
@@ -113,6 +113,7 @@ import org.apache.hadoop.yarn.api.records.ReservationRequest;
 import org.apache.hadoop.yarn.api.records.ReservationRequestInterpreter;
 import org.apache.hadoop.yarn.api.records.ReservationRequests;
 import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.api.records.URL;
 import org.apache.hadoop.yarn.api.records.YarnApplicationState;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.exceptions.YarnException;
@@ -373,7 +374,7 @@ public class RMWebServices extends WebServices {
     if (sched == null) {
       throw new NotFoundException("Null ResourceScheduler instance");
     }
-    NodeId nid = ConverterUtils.toNodeId(nodeId);
+    NodeId nid = NodeId.fromString(nodeId);
     RMNode ni = this.rm.getRMContext().getRMNodes().get(nid);
     boolean isInactive = false;
     if (ni == null) {
@@ -1467,9 +1468,7 @@ public class RMWebServices extends WebServices {
     String error =
         "Could not parse application id " + newApp.getApplicationId();
     try {
-      appid =
-          ConverterUtils.toApplicationId(recordFactory,
-            newApp.getApplicationId());
+      appid = ApplicationId.fromString(newApp.getApplicationId());
     } catch (Exception e) {
       throw new BadRequestException(error);
     }
@@ -1553,7 +1552,7 @@ public class RMWebServices extends WebServices {
       LocalResourceInfo l = entry.getValue();
       LocalResource lr =
           LocalResource.newInstance(
-            ConverterUtils.getYarnUrlFromURI(l.getUrl()), l.getType(),
+              URL.fromURI(l.getUrl()), l.getType(),
             l.getVisibility(), l.getSize(), l.getTimestamp());
       hlr.put(entry.getKey(), lr);
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c77a1095/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/AppAttemptInfo.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/AppAttemptInfo.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/AppAttemptInfo.java
index e8c8bca..55bf999 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/AppAttemptInfo.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/AppAttemptInfo.java
@@ -67,7 +67,7 @@ public class AppAttemptInfo {
         this.nodeId = masterContainer.getNodeId().toString();
         this.logsLink = WebAppUtils.getRunningLogURL(schemePrefix
             + masterContainer.getNodeHttpAddress(),
-            ConverterUtils.toString(masterContainer.getId()), user);
+            masterContainer.getId().toString(), user);
 
         nodesBlacklistedBySystem =
             StringUtils.join(attempt.getAMBlacklistManager()

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c77a1095/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 63b601d..c5c02a8 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
@@ -165,8 +165,7 @@ public class AppInfo {
             this.amContainerLogsExist = true;
             this.amContainerLogs = WebAppUtils.getRunningLogURL(
                 schemePrefix + masterContainer.getNodeHttpAddress(),
-                ConverterUtils.toString(masterContainer.getId()),
-                app.getUser());
+                masterContainer.getId().toString(), app.getUser());
             this.amHostHttpAddress = masterContainer.getNodeHttpAddress();
           }
           

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c77a1095/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMAdminService.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/TestRMAdminService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMAdminService.java
index b109639..af342df 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMAdminService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMAdminService.java
@@ -218,7 +218,7 @@ public class TestRMAdminService {
       fail("Should not get any exceptions");
     }
 
-    NodeId nid = ConverterUtils.toNodeId("h1:1234");
+    NodeId nid = NodeId.fromString("h1:1234");
     RMNode ni = rm.getRMContext().getRMNodes().get(nid);
     Resource resource = ni.getTotalCapability();
     Assert.assertEquals("<memory:5120, vCores:5>", resource.toString());
@@ -257,7 +257,7 @@ public class TestRMAdminService {
       fail("Should not get any exceptions");
     }
 
-    NodeId nid = ConverterUtils.toNodeId("h1:1234");
+    NodeId nid = NodeId.fromString("h1:1234");
     RMNode ni = rm.getRMContext().getRMNodes().get(nid);
     Resource resource = ni.getTotalCapability();
     Assert.assertEquals("<memory:2048, vCores:2>", resource.toString());
@@ -307,7 +307,7 @@ public class TestRMAdminService {
       fail("Should not get any exceptions");
     }
 
-    NodeId nid = ConverterUtils.toNodeId("h1:1234");
+    NodeId nid = NodeId.fromString("h1:1234");
     RMNode ni = rm.getRMContext().getRMNodes().get(nid);
     Resource resource = ni.getTotalCapability();
     Assert.assertEquals("<memory:2048, vCores:2>", resource.toString());
@@ -355,7 +355,7 @@ public class TestRMAdminService {
       fail("Should not get any exceptions");
     }
 
-    NodeId nid = ConverterUtils.toNodeId("h1:1234");
+    NodeId nid = NodeId.fromString("h1:1234");
     RMNode ni = rm.getRMContext().getRMNodes().get(nid);
     Resource resource = ni.getTotalCapability();
     Assert.assertEquals("<memory:5120, vCores:5>", resource.toString());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c77a1095/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/RMStateStoreTestBase.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/RMStateStoreTestBase.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/RMStateStoreTestBase.java
index d46ed05..758bbae 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/RMStateStoreTestBase.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/RMStateStoreTestBase.java
@@ -183,7 +183,7 @@ public class RMStateStoreTestBase {
     RMAppAttemptMetrics mockRmAppAttemptMetrics = 
         mock(RMAppAttemptMetrics.class);
     Container container = new ContainerPBImpl();
-    container.setId(ConverterUtils.toContainerId(containerIdStr));
+    container.setId(ContainerId.fromString(containerIdStr));
     RMAppAttempt mockAttempt = mock(RMAppAttempt.class);
     when(mockAttempt.getAppAttemptId()).thenReturn(attemptId);
     when(mockAttempt.getMasterContainer()).thenReturn(container);
@@ -227,8 +227,8 @@ public class RMStateStoreTestBase {
     ClientToAMTokenSecretManagerInRM clientToAMTokenMgr =
         new ClientToAMTokenSecretManagerInRM();
 
-    ApplicationAttemptId attemptId1 = ConverterUtils
-        .toApplicationAttemptId("appattempt_1352994193343_0001_000001");
+    ApplicationAttemptId attemptId1 = ApplicationAttemptId.fromString(
+        "appattempt_1352994193343_0001_000001");
     ApplicationId appId1 = attemptId1.getApplicationId();
     storeApp(store, appId1, submitTime, startTime);
     verifier.afterStoreApp(store, appId1);
@@ -245,8 +245,8 @@ public class RMStateStoreTestBase {
         .getMasterContainer().getId();
 
     String appAttemptIdStr2 = "appattempt_1352994193343_0001_000002";
-    ApplicationAttemptId attemptId2 =
-        ConverterUtils.toApplicationAttemptId(appAttemptIdStr2);
+    ApplicationAttemptId attemptId2 = ApplicationAttemptId.fromString(
+        appAttemptIdStr2);
 
     // create application token and client token key for attempt2
     Token<AMRMTokenIdentifier> appAttemptToken2 =
@@ -259,8 +259,8 @@ public class RMStateStoreTestBase {
         appAttemptToken2, clientTokenKey2, dispatcher)
         .getMasterContainer().getId();
 
-    ApplicationAttemptId attemptIdRemoved = ConverterUtils
-        .toApplicationAttemptId("appattempt_1352994193343_0002_000001");
+    ApplicationAttemptId attemptIdRemoved = ApplicationAttemptId.fromString(
+        "appattempt_1352994193343_0002_000001");
     ApplicationId appIdRemoved = attemptIdRemoved.getApplicationId();
     storeApp(store, appIdRemoved, submitTime, startTime);
     storeAttempt(store, attemptIdRemoved,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c77a1095/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 a51ccb5..61088e1 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
@@ -183,7 +183,7 @@ public class TestFSRMStateStore extends RMStateStoreTestBase {
               (FileSystemRMStateStore) fsTester.getRMStateStore();
       String appAttemptIdStr3 = "appattempt_1352994193343_0001_000003";
       ApplicationAttemptId attemptId3 =
-              ConverterUtils.toApplicationAttemptId(appAttemptIdStr3);
+          ApplicationAttemptId.fromString(appAttemptIdStr3);
       Path appDir =
               fsTester.store.getAppDir(attemptId3.getApplicationId().toString());
       Path tempAppAttemptFile =
@@ -364,7 +364,7 @@ public class TestFSRMStateStore extends RMStateStoreTestBase {
     // imitate appAttemptFile1 is still .new, but old one is deleted
     String appAttemptIdStr1 = "appattempt_1352994193343_0001_000001";
     ApplicationAttemptId attemptId1 =
-        ConverterUtils.toApplicationAttemptId(appAttemptIdStr1);
+        ApplicationAttemptId.fromString(appAttemptIdStr1);
     Path appDir =
             fsTester.store.getAppDir(attemptId1.getApplicationId().toString());
     Path appAttemptFile1 =

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c77a1095/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/TestZKRMStateStore.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/TestZKRMStateStore.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/TestZKRMStateStore.java
index 7df31cf..19d3064 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/TestZKRMStateStore.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/TestZKRMStateStore.java
@@ -34,6 +34,7 @@ import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
 import org.apache.hadoop.yarn.api.records.Container;
+import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
 import org.apache.hadoop.yarn.api.records.impl.pb.ApplicationSubmissionContextPBImpl;
 import org.apache.hadoop.yarn.api.records.impl.pb.ContainerPBImpl;
@@ -399,14 +400,14 @@ public class TestZKRMStateStore extends RMStateStoreTestBase {
     // Add a new attempt
     ClientToAMTokenSecretManagerInRM clientToAMTokenMgr =
             new ClientToAMTokenSecretManagerInRM();
-    ApplicationAttemptId attemptId = ConverterUtils
-            .toApplicationAttemptId("appattempt_1234567894321_0001_000001");
+    ApplicationAttemptId attemptId = ApplicationAttemptId.fromString(
+        "appattempt_1234567894321_0001_000001");
     SecretKey clientTokenMasterKey =
                 clientToAMTokenMgr.createMasterKey(attemptId);
     RMAppAttemptMetrics mockRmAppAttemptMetrics = 
          mock(RMAppAttemptMetrics.class);
     Container container = new ContainerPBImpl();
-    container.setId(ConverterUtils.toContainerId("container_1234567891234_0001_01_000001"));
+    container.setId(ContainerId.fromString("container_1234567891234_0001_01_000001"));
     RMAppAttempt mockAttempt = mock(RMAppAttempt.class);
     when(mockAttempt.getAppAttemptId()).thenReturn(attemptId);
     when(mockAttempt.getMasterContainer()).thenReturn(container);
@@ -491,8 +492,8 @@ public class TestZKRMStateStore extends RMStateStoreTestBase {
     TestDispatcher dispatcher = new TestDispatcher();
     store.setRMDispatcher(dispatcher);
 
-    ApplicationAttemptId attemptIdRemoved = ConverterUtils
-        .toApplicationAttemptId("appattempt_1352994193343_0002_000001");
+    ApplicationAttemptId attemptIdRemoved = ApplicationAttemptId.fromString(
+        "appattempt_1352994193343_0002_000001");
     ApplicationId appIdRemoved = attemptIdRemoved.getApplicationId();
     storeApp(store, appIdRemoved, submitTime, startTime);
     storeAttempt(store, attemptIdRemoved,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c77a1095/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesAppsModification.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/TestRMWebServicesAppsModification.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesAppsModification.java
index c7ef8fa..682ed75 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesAppsModification.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesAppsModification.java
@@ -57,6 +57,7 @@ import org.apache.hadoop.security.Credentials;
 import org.apache.hadoop.security.authentication.server.AuthenticationFilter;
 import org.apache.hadoop.security.authentication.server.PseudoAuthenticationHandler;
 import org.apache.hadoop.yarn.api.records.ApplicationAccessType;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
 import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
 import org.apache.hadoop.yarn.api.records.LocalResource;
@@ -65,6 +66,7 @@ import org.apache.hadoop.yarn.api.records.LocalResourceVisibility;
 import org.apache.hadoop.yarn.api.records.LogAggregationContext;
 import org.apache.hadoop.yarn.api.records.QueueACL;
 import org.apache.hadoop.yarn.api.records.ReservationId;
+import org.apache.hadoop.yarn.api.records.URL;
 import org.apache.hadoop.yarn.api.records.YarnApplicationState;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.server.resourcemanager.MockNM;
@@ -859,7 +861,7 @@ public class TestRMWebServicesAppsModification extends JerseyTestBase {
 
     RMApp app =
         rm.getRMContext().getRMApps()
-          .get(ConverterUtils.toApplicationId(appId));
+          .get(ApplicationId.fromString(appId));
     assertEquals(appName, app.getName());
     assertEquals(webserviceUserName, app.getUser());
     assertEquals(2, app.getMaxAppAttempts());
@@ -877,8 +879,7 @@ public class TestRMWebServicesAppsModification extends JerseyTestBase {
     Map<String, LocalResource> appLRs = ctx.getLocalResources();
     assertTrue(appLRs.containsKey(lrKey));
     LocalResource exampleLR = appLRs.get(lrKey);
-    assertEquals(ConverterUtils.getYarnUrlFromURI(y.getUrl()),
-      exampleLR.getResource());
+    assertEquals(URL.fromURI(y.getUrl()), exampleLR.getResource());
     assertEquals(y.getSize(), exampleLR.getSize());
     assertEquals(y.getTimestamp(), exampleLR.getTimestamp());
     assertEquals(y.getType(), exampleLR.getType());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c77a1095/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesDelegationTokenAuthentication.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/TestRMWebServicesDelegationTokenAuthentication.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesDelegationTokenAuthentication.java
index 36e24ec..4e26bd1 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesDelegationTokenAuthentication.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesDelegationTokenAuthentication.java
@@ -50,6 +50,7 @@ import org.apache.hadoop.security.authentication.server.AuthenticationFilter;
 import org.apache.hadoop.security.authentication.server.KerberosAuthenticationHandler;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.security.token.delegation.web.DelegationTokenAuthenticator;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.security.client.RMDelegationTokenIdentifier;
 import org.apache.hadoop.yarn.server.resourcemanager.MockRM;
@@ -236,11 +237,11 @@ public class TestRMWebServicesDelegationTokenAuthentication {
 
     boolean appExists =
         rm.getRMContext().getRMApps()
-          .containsKey(ConverterUtils.toApplicationId(appid));
+          .containsKey(ApplicationId.fromString(appid));
     assertTrue(appExists);
     RMApp actualApp =
         rm.getRMContext().getRMApps()
-          .get(ConverterUtils.toApplicationId(appid));
+          .get(ApplicationId.fromString(appid));
     String owner = actualApp.getUser();
     assertEquals("client", owner);
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c77a1095/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebappAuthentication.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/TestRMWebappAuthentication.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebappAuthentication.java
index 2f6a022..249e825 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebappAuthentication.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebappAuthentication.java
@@ -37,6 +37,7 @@ import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
 import org.apache.hadoop.minikdc.MiniKdc;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.authentication.KerberosTestUtils;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.server.resourcemanager.MockRM;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
@@ -236,11 +237,11 @@ public class TestRMWebappAuthentication {
     assertEquals(Status.ACCEPTED.getStatusCode(), conn.getResponseCode());
     boolean appExists =
         rm.getRMContext().getRMApps()
-          .containsKey(ConverterUtils.toApplicationId(appid));
+          .containsKey(ApplicationId.fromString(appid));
     assertTrue(appExists);
     RMApp actualApp =
         rm.getRMContext().getRMApps()
-          .get(ConverterUtils.toApplicationId(appid));
+          .get(ApplicationId.fromString(appid));
     String owner = actualApp.getUser();
     assertEquals(
       rm.getConfig().get(CommonConfigurationKeys.HADOOP_HTTP_STATIC_USER,
@@ -259,11 +260,11 @@ public class TestRMWebappAuthentication {
     conn.getInputStream();
     appExists =
         rm.getRMContext().getRMApps()
-          .containsKey(ConverterUtils.toApplicationId(appid));
+          .containsKey(ApplicationId.fromString(appid));
     assertTrue(appExists);
     actualApp =
         rm.getRMContext().getRMApps()
-          .get(ConverterUtils.toApplicationId(appid));
+          .get(ApplicationId.fromString(appid));
     owner = actualApp.getUser();
     assertEquals("client", owner);
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c77a1095/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timeline-pluginstorage/src/main/java/org/apache/hadoop/yarn/server/timeline/EntityGroupFSTimelineStore.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timeline-pluginstorage/src/main/java/org/apache/hadoop/yarn/server/timeline/EntityGroupFSTimelineStore.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timeline-pluginstorage/src/main/java/org/apache/hadoop/yarn/server/timeline/EntityGroupFSTimelineStore.java
index 231ca72..958b54e 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timeline-pluginstorage/src/main/java/org/apache/hadoop/yarn/server/timeline/EntityGroupFSTimelineStore.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timeline-pluginstorage/src/main/java/org/apache/hadoop/yarn/server/timeline/EntityGroupFSTimelineStore.java
@@ -481,7 +481,7 @@ public class EntityGroupFSTimelineStore extends CompositeService
     ApplicationId appId = null;
     if (appIdStr.startsWith(ApplicationId.appIdStrPrefix)) {
       try {
-        appId = ConverterUtils.toApplicationId(appIdStr);
+        appId = ApplicationId.fromString(appIdStr);
       } catch (IllegalArgumentException e) {
         appId = null;
       }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c77a1095/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timeline-pluginstorage/src/test/java/org/apache/hadoop/yarn/server/timeline/EntityGroupPlugInForTest.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timeline-pluginstorage/src/test/java/org/apache/hadoop/yarn/server/timeline/EntityGroupPlugInForTest.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timeline-pluginstorage/src/test/java/org/apache/hadoop/yarn/server/timeline/EntityGroupPlugInForTest.java
index db241a8..884b5cd 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timeline-pluginstorage/src/test/java/org/apache/hadoop/yarn/server/timeline/EntityGroupPlugInForTest.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timeline-pluginstorage/src/test/java/org/apache/hadoop/yarn/server/timeline/EntityGroupPlugInForTest.java
@@ -34,15 +34,16 @@ class EntityGroupPlugInForTest extends TimelineEntityGroupPlugin {
   public Set<TimelineEntityGroupId> getTimelineEntityGroupId(String entityType,
       NameValuePair primaryFilter,
       Collection<NameValuePair> secondaryFilters) {
-    ApplicationId appId
-        = ConverterUtils.toApplicationId(primaryFilter.getValue().toString());
+    ApplicationId appId = ApplicationId.fromString(
+        primaryFilter.getValue().toString());
     return Sets.newHashSet(getStandardTimelineGroupId(appId));
   }
 
   @Override
   public Set<TimelineEntityGroupId> getTimelineEntityGroupId(String entityId,
       String entityType) {
-    ApplicationId appId = ConverterUtils.toApplicationId(entityId);
+    ApplicationId appId = ApplicationId.fromString(
+        entityId);
     return Sets.newHashSet(getStandardTimelineGroupId(appId));
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c77a1095/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timeline-pluginstorage/src/test/java/org/apache/hadoop/yarn/server/timeline/TestEntityGroupFSTimelineStore.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timeline-pluginstorage/src/test/java/org/apache/hadoop/yarn/server/timeline/TestEntityGroupFSTimelineStore.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timeline-pluginstorage/src/test/java/org/apache/hadoop/yarn/server/timeline/TestEntityGroupFSTimelineStore.java
index d6baab6..1c12f36 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timeline-pluginstorage/src/test/java/org/apache/hadoop/yarn/server/timeline/TestEntityGroupFSTimelineStore.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timeline-pluginstorage/src/test/java/org/apache/hadoop/yarn/server/timeline/TestEntityGroupFSTimelineStore.java
@@ -68,7 +68,7 @@ public class TestEntityGroupFSTimelineStore extends TimelineStoreTestUtils {
 
   private static final String SAMPLE_APP_PREFIX_CACHE_TEST = "1234_000";
   private static final int CACHE_TEST_CACHE_SIZE = 5;
-
+  
   private static final String TEST_SUMMARY_LOG_FILE_NAME
       = EntityGroupFSTimelineStore.SUMMARY_LOG_PREFIX + "test";
   private static final String TEST_DOMAIN_LOG_FILE_NAME
@@ -117,7 +117,7 @@ public class TestEntityGroupFSTimelineStore extends TimelineStoreTestUtils {
 
     sampleAppIds = new ArrayList<>(CACHE_TEST_CACHE_SIZE + 1);
     for (int i = 0; i < CACHE_TEST_CACHE_SIZE + 1; i++) {
-      ApplicationId appId = ConverterUtils.toApplicationId(
+      ApplicationId appId = ApplicationId.fromString(
           ConverterUtils.APPLICATION_PREFIX + "_" + SAMPLE_APP_PREFIX_CACHE_TEST
               + i);
       sampleAppIds.add(appId);


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


[12/39] hadoop git commit: YARN-5122. "yarn logs" for running containers should print an explicit footer saying that the log may be incomplete. Contributed by Jian He.

Posted by ae...@apache.org.
YARN-5122. "yarn logs" for running containers should print an explicit footer saying that the log may be incomplete. 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/d6284428
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/d6284428
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/d6284428

Branch: refs/heads/HDFS-1312
Commit: d6284428e8bd09a06cf18ff81a7064d744ef47d0
Parents: 098ae11
Author: Varun Vasudev <vv...@apache.org>
Authored: Wed Jun 15 12:35:17 2016 +0530
Committer: Varun Vasudev <vv...@apache.org>
Committed: Wed Jun 15 12:35:17 2016 +0530

----------------------------------------------------------------------
 .../src/main/java/org/apache/hadoop/yarn/client/cli/LogsCLI.java | 4 +++-
 1 file changed, 3 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/d6284428/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/LogsCLI.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/LogsCLI.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/LogsCLI.java
index 4fdb57b..c909402 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/LogsCLI.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/LogsCLI.java
@@ -412,7 +412,9 @@ public class LogsCLI extends Configured implements Tool {
                 .queryParam("size", Long.toString(request.getBytes()))
                 .accept(MediaType.TEXT_PLAIN).get(ClientResponse.class);
           out.println(response.getEntity(String.class));
-          out.println("End of LogType:" + logFile);
+          out.println("End of LogType:" + logFile + ". This log file belongs"
+              + " to a running container (" + containerIdStr + ") and so may"
+              + " not be complete.");
           out.flush();
         } catch (ClientHandlerException | UniformInterfaceException ex) {
           System.err.println("Can not find the log file:" + logFile


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


[10/39] hadoop git commit: HDFS-9016. Display upgrade domain information in fsck. (mingma)

Posted by ae...@apache.org.
HDFS-9016. Display upgrade domain information in fsck. (mingma)


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

Branch: refs/heads/HDFS-1312
Commit: 7d521a29eed62c4329b16034375bd5fb747a92a9
Parents: e2f6409
Author: Ming Ma <mi...@apache.org>
Authored: Tue Jun 14 20:05:50 2016 -0700
Committer: Ming Ma <mi...@apache.org>
Committed: Tue Jun 14 20:05:50 2016 -0700

----------------------------------------------------------------------
 .../hdfs/server/namenode/NamenodeFsck.java      | 24 +++++--
 .../org/apache/hadoop/hdfs/tools/DFSck.java     | 16 +++--
 .../src/site/markdown/HDFSCommands.md           |  3 +-
 .../hadoop/hdfs/server/namenode/TestFsck.java   | 74 +++++++++++++++++++-
 4 files changed, 103 insertions(+), 14 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/7d521a29/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 a85c68c..d7c9a78 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
@@ -118,6 +118,7 @@ public class NamenodeFsck implements DataEncryptionKeyFactory {
   public static final String DECOMMISSIONED_STATUS = "is DECOMMISSIONED";
   public static final String NONEXISTENT_STATUS = "does not exist";
   public static final String FAILURE_STATUS = "FAILED";
+  public static final String UNDEFINED = "undefined";
 
   private final NameNode namenode;
   private final BlockManager blockManager;
@@ -141,6 +142,7 @@ public class NamenodeFsck implements DataEncryptionKeyFactory {
   private boolean showCorruptFileBlocks = false;
 
   private boolean showReplicaDetails = false;
+  private boolean showUpgradeDomains = false;
   private long staleInterval;
   private Tracer tracer;
 
@@ -222,11 +224,15 @@ public class NamenodeFsck implements DataEncryptionKeyFactory {
       else if (key.equals("racks")) { this.showRacks = true; }
       else if (key.equals("replicadetails")) {
         this.showReplicaDetails = 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")) {
+      } else if (key.equals("upgradedomains")) {
+        this.showUpgradeDomains = 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")) {
         this.showCorruptFileBlocks = true;
       } else if (key.equals("startblockafter")) {
         this.currentCookie[0] = pmap.get("startblockafter")[0];
@@ -550,7 +556,8 @@ public class NamenodeFsck implements DataEncryptionKeyFactory {
    * For striped block group, display info of each internal block.
    */
   private String getReplicaInfo(BlockInfo storedBlock) {
-    if (!(showLocations || showRacks || showReplicaDetails)) {
+    if (!(showLocations || showRacks || showReplicaDetails ||
+        showUpgradeDomains)) {
       return "";
     }
     final boolean isComplete = storedBlock.isComplete();
@@ -568,6 +575,11 @@ public class NamenodeFsck implements DataEncryptionKeyFactory {
         sb.append(new DatanodeInfoWithStorage(dnDesc, storage.getStorageID(),
             storage.getStorageType()));
       }
+      if (showUpgradeDomains) {
+        String upgradeDomain = (dnDesc.getUpgradeDomain() != null) ?
+            dnDesc.getUpgradeDomain() : UNDEFINED;
+        sb.append("(ud=" + upgradeDomain +")");
+      }
       if (showReplicaDetails) {
         Collection<DatanodeDescriptor> corruptReplicas =
             blockManager.getCorruptReplicas(storedBlock);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7d521a29/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 ab689ff..9cf234a 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
@@ -77,7 +77,8 @@ public class DFSck extends Configured implements Tool {
   private static final String USAGE = "Usage: hdfs fsck <path> "
       + "[-list-corruptfileblocks | "
       + "[-move | -delete | -openforwrite] "
-      + "[-files [-blocks [-locations | -racks | -replicaDetails]]]] "
+      + "[-files [-blocks [-locations | -racks | -replicaDetails | " +
+          "-upgradedomains]]]] "
       + "[-includeSnapshots] [-showprogress] "
       + "[-storagepolicies] [-blockId <blk_Id>]\n"
       + "\t<path>\tstart checking from this path\n"
@@ -95,6 +96,8 @@ public class DFSck extends Configured implements Tool {
       + "\t-files -blocks -racks" 
       + "\tprint out network topology for data-node locations\n"
       + "\t-files -blocks -replicaDetails\tprint out each replica details \n"
+      + "\t-files -blocks -upgradedomains\tprint out upgrade domains for " +
+          "every block\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"
@@ -272,10 +275,13 @@ public class DFSck extends Configured implements Tool {
       else if (args[idx].equals("-racks")) { url.append("&racks=1"); }
       else if (args[idx].equals("-replicaDetails")) {
         url.append("&replicadetails=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")) {
+      } else if (args[idx].equals("-upgradedomains")) {
+        url.append("&upgradedomains=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");
         doListCorruptFileBlocks = true;
       } else if (args[idx].equals("-includeSnapshots")) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7d521a29/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSCommands.md
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSCommands.md b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSCommands.md
index 7c94e92..5bb1a87 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSCommands.md
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSCommands.md
@@ -117,7 +117,7 @@ Usage:
        hdfs fsck <path>
               [-list-corruptfileblocks |
               [-move | -delete | -openforwrite]
-              [-files [-blocks [-locations | -racks | -replicaDetails]]]
+              [-files [-blocks [-locations | -racks | -replicaDetails | -upgradedomains]]]
               [-includeSnapshots] [-showprogress]
               [-storagepolicies] [-blockId <blk_Id>]
 
@@ -130,6 +130,7 @@ Usage:
 | `-files` `-blocks` `-locations` | Print out locations for every block. |
 | `-files` `-blocks` `-racks` | Print out network topology for data-node locations. |
 | `-files` `-blocks` `-replicaDetails` | Print out each replica details. |
+| `-files` `-blocks` `-upgradedomains` | Print out upgrade domains for every block. |
 | `-includeSnapshots` | Include snapshot data if the given path indicates a snapshottable directory or there are snapshottable directories under it. |
 | `-list-corruptfileblocks` | Print out list of missing blocks and files they belong to. |
 | `-move` | Move corrupted files to /lost+found. |

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7d521a29/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 7cb6edc..73ad885 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
@@ -24,9 +24,7 @@ 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.any;
 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;
@@ -81,6 +79,8 @@ import org.apache.hadoop.fs.StorageType;
 import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.CorruptFileBlocks;
+import org.apache.hadoop.hdfs.protocol.DatanodeAdminProperties;
+import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
@@ -90,13 +90,16 @@ import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockCollection;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
+import org.apache.hadoop.hdfs.server.blockmanagement.CombinedHostFileManager;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeManager;
+import org.apache.hadoop.hdfs.server.blockmanagement.HostConfigManager;
 import org.apache.hadoop.hdfs.server.namenode.NamenodeFsck.Result;
 import org.apache.hadoop.hdfs.server.namenode.NamenodeFsck.ReplicationResult;
 import org.apache.hadoop.hdfs.server.namenode.NamenodeFsck.ErasureCodingResult;
 import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocols;
 import org.apache.hadoop.hdfs.tools.DFSck;
+import org.apache.hadoop.hdfs.util.HostsFileWriter;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.net.NetworkTopology;
 import org.apache.hadoop.security.AccessControlException;
@@ -2008,4 +2011,71 @@ public class TestFsck {
     }
   }
 
+  @Test(timeout = 60000)
+  public void testFsckUpgradeDomain() throws Exception {
+    testUpgradeDomain(false, false);
+    testUpgradeDomain(false, true);
+    testUpgradeDomain(true, false);
+    testUpgradeDomain(true, true);
+  }
+
+  private void testUpgradeDomain(boolean defineUpgradeDomain,
+      boolean displayUpgradeDomain) throws Exception {
+    final short replFactor = 1;
+    final short numDN = 1;
+    final long blockSize = 512;
+    final long fileSize = 1024;
+    final String upgradeDomain = "ud1";
+    final String[] racks = {"/rack1"};
+    final String[] hosts = {"127.0.0.1"};
+    HostsFileWriter hostsFileWriter = new HostsFileWriter();
+    Configuration conf = new Configuration();
+    conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, blockSize);
+    conf.setInt(DFSConfigKeys.DFS_REPLICATION_KEY, replFactor);
+    if (defineUpgradeDomain) {
+      conf.setClass(DFSConfigKeys.DFS_NAMENODE_HOSTS_PROVIDER_CLASSNAME_KEY,
+          CombinedHostFileManager.class, HostConfigManager.class);
+      hostsFileWriter.initialize(conf, "temp/fsckupgradedomain");
+    }
+
+    MiniDFSCluster cluster;
+    DistributedFileSystem dfs;
+    cluster = new MiniDFSCluster.Builder(conf).numDataNodes(numDN).
+        hosts(hosts).racks(racks).build();
+    cluster.waitClusterUp();
+    dfs = cluster.getFileSystem();
+
+    // Configure the upgrade domain on the datanode
+    if (defineUpgradeDomain) {
+      DatanodeAdminProperties dnProp = new DatanodeAdminProperties();
+      DatanodeID datanodeID = cluster.getDataNodes().get(0).getDatanodeId();
+      dnProp.setHostName(datanodeID.getHostName());
+      dnProp.setPort(datanodeID.getXferPort());
+      dnProp.setUpgradeDomain(upgradeDomain);
+      hostsFileWriter.initIncludeHosts(new DatanodeAdminProperties[]{dnProp});
+      cluster.getFileSystem().refreshNodes();
+    }
+
+    // create files
+    final String testFile = new String("/testfile");
+    final Path path = new Path(testFile);
+    DFSTestUtil.createFile(dfs, path, fileSize, replFactor, 1000L);
+    DFSTestUtil.waitReplication(dfs, path, replFactor);
+    try {
+      String fsckOut = runFsck(conf, 0, true, testFile, "-files", "-blocks",
+          displayUpgradeDomain ? "-upgradedomains" : "-locations");
+      assertTrue(fsckOut.contains(NamenodeFsck.HEALTHY_STATUS));
+      String udValue = defineUpgradeDomain ? upgradeDomain :
+          NamenodeFsck.UNDEFINED;
+      assertEquals(displayUpgradeDomain,
+          fsckOut.contains("(ud=" + udValue + ")"));
+    } finally {
+      if (defineUpgradeDomain) {
+        hostsFileWriter.cleanup();
+      }
+      if (cluster != null) {
+        cluster.shutdown();
+      }
+    }
+  }
 }


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


[24/39] hadoop git commit: HADOOP-3733. "s3x:" URLs break when Secret Key contains a slash, even if encoded. Contributed by Steve Loughran.

Posted by ae...@apache.org.
HADOOP-3733. "s3x:" URLs break when Secret Key contains a slash, even if encoded. Contributed by Steve Loughran.


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

Branch: refs/heads/HDFS-1312
Commit: 4aefe119a0203c03cdc893dcb3330fd37f26f0ee
Parents: e983eaf
Author: Ravi Prakash <ra...@altiscale.com>
Authored: Thu Jun 16 11:13:35 2016 -0700
Committer: Ravi Prakash <ra...@altiscale.com>
Committed: Thu Jun 16 11:13:35 2016 -0700

----------------------------------------------------------------------
 .../hadoop-aws/dev-support/findbugs-exclude.xml |   7 +
 .../org/apache/hadoop/fs/s3/S3Credentials.java  |  31 +-
 .../org/apache/hadoop/fs/s3/S3FileSystem.java   |  20 +-
 .../org/apache/hadoop/fs/s3a/S3AFileSystem.java |  79 ++----
 .../java/org/apache/hadoop/fs/s3a/S3AUtils.java |  46 +++
 .../hadoop/fs/s3native/NativeS3FileSystem.java  |  19 +-
 .../hadoop/fs/s3native/S3xLoginHelper.java      | 283 +++++++++++++++++++
 .../src/site/markdown/tools/hadoop-aws/index.md |   5 +-
 .../apache/hadoop/fs/s3/TestS3FileSystem.java   |  24 +-
 .../hadoop/fs/s3a/TestS3AConfiguration.java     |  55 ++--
 .../hadoop/fs/s3a/TestS3ACredentialsInURL.java  | 153 ++++++++++
 .../hadoop/fs/s3native/TestS3xLoginHelper.java  | 197 +++++++++++++
 12 files changed, 801 insertions(+), 118 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/4aefe119/hadoop-tools/hadoop-aws/dev-support/findbugs-exclude.xml
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/dev-support/findbugs-exclude.xml b/hadoop-tools/hadoop-aws/dev-support/findbugs-exclude.xml
index 2b4160a..ffb0a79 100644
--- a/hadoop-tools/hadoop-aws/dev-support/findbugs-exclude.xml
+++ b/hadoop-tools/hadoop-aws/dev-support/findbugs-exclude.xml
@@ -15,6 +15,13 @@
    limitations under the License.
 -->
 <FindBugsFilter>
+
+  <!-- same code as in FileSystem is triggering the same warning. -->
+  <Match>
+    <Class name="org.apache.hadoop.fs.s3native.S3xLoginHelper" />
+    <Method name="checkPath" />
+    <Bug pattern="ES_COMPARING_STRINGS_WITH_EQ" />
+  </Match>
   <!-- S3n warnings about malicious code aren't that relevant given its limited future. -->
   <Match>
     <Class name="org.apache.hadoop.fs.s3.INode" />

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4aefe119/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3/S3Credentials.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3/S3Credentials.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3/S3Credentials.java
index 5ab352a..3951a08 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3/S3Credentials.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3/S3Credentials.java
@@ -24,6 +24,7 @@ import java.net.URI;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.s3native.S3xLoginHelper;
 
 /**
  * <p>
@@ -49,18 +50,12 @@ public class S3Credentials {
     if (uri.getHost() == null) {
       throw new IllegalArgumentException("Invalid hostname in URI " + uri);
     }
-    
-    String userInfo = uri.getUserInfo();
-    if (userInfo != null) {
-      int index = userInfo.indexOf(':');
-      if (index != -1) {
-        accessKey = userInfo.substring(0, index);
-        secretAccessKey = userInfo.substring(index + 1);
-      } else {
-        accessKey = userInfo;
-      }
+    S3xLoginHelper.Login login =
+        S3xLoginHelper.extractLoginDetailsWithWarnings(uri);
+    if (login.hasLogin()) {
+      accessKey = login.getUser();
+      secretAccessKey = login.getPassword();
     }
-    
     String scheme = uri.getScheme();
     String accessKeyProperty = String.format("fs.%s.awsAccessKeyId", scheme);
     String secretAccessKeyProperty =
@@ -77,24 +72,20 @@ public class S3Credentials {
     if (accessKey == null && secretAccessKey == null) {
       throw new IllegalArgumentException("AWS " +
                                          "Access Key ID and Secret Access " +
-                                         "Key must be specified as the " +
-                                         "username or password " +
-                                         "(respectively) of a " + scheme +
-                                         " URL, or by setting the " +
-                                         accessKeyProperty + " or " +
+                                         "Key must be specified " +
+                                         "by setting the " +
+                                         accessKeyProperty + " and " +
                                          secretAccessKeyProperty +
                                          " properties (respectively).");
     } else if (accessKey == null) {
       throw new IllegalArgumentException("AWS " +
                                          "Access Key ID must be specified " +
-                                         "as the username of a " + scheme +
-                                         " URL, or by setting the " +
+                                         "by setting the " +
                                          accessKeyProperty + " property.");
     } else if (secretAccessKey == null) {
       throw new IllegalArgumentException("AWS " +
                                          "Secret Access Key must be " +
-                                         "specified as the password of a " +
-                                         scheme + " URL, or by setting the " +
+                                         "specified by setting the " +
                                          secretAccessKeyProperty +
                                          " property.");       
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4aefe119/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3/S3FileSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3/S3FileSystem.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3/S3FileSystem.java
index e5147a3..6a49d1a 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3/S3FileSystem.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3/S3FileSystem.java
@@ -39,6 +39,7 @@ import org.apache.hadoop.fs.ParentNotDirectoryException;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.s3native.NativeS3FileSystem;
+import org.apache.hadoop.fs.s3native.S3xLoginHelper;
 import org.apache.hadoop.io.retry.RetryPolicies;
 import org.apache.hadoop.io.retry.RetryPolicy;
 import org.apache.hadoop.io.retry.RetryProxy;
@@ -91,7 +92,7 @@ public class S3FileSystem extends FileSystem {
     }
     store.initialize(uri, conf);
     setConf(conf);
-    this.uri = URI.create(uri.getScheme() + "://" + uri.getAuthority());
+    this.uri = S3xLoginHelper.buildFSURI(uri);
     this.workingDir =
       new Path("/user", System.getProperty("user.name")).makeQualified(this);
   }
@@ -135,6 +136,23 @@ public class S3FileSystem extends FileSystem {
   }
 
   /**
+   * Check that a Path belongs to this FileSystem.
+   * Unlike the superclass, this version does not look at authority,
+   * only hostnames.
+   * @param path to check
+   * @throws IllegalArgumentException if there is an FS mismatch
+   */
+  @Override
+  protected void checkPath(Path path) {
+    S3xLoginHelper.checkPath(getConf(), getUri(), path, getDefaultPort());
+  }
+
+  @Override
+  protected URI canonicalizeUri(URI rawUri) {
+    return S3xLoginHelper.canonicalizeUri(rawUri, getDefaultPort());
+  }
+
+  /**
    * @param permission Currently ignored.
    */
   @Override

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4aefe119/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java
index 9af0a99..63bfb4f 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java
@@ -78,10 +78,11 @@ import org.apache.hadoop.fs.PathFilter;
 import org.apache.hadoop.fs.RemoteIterator;
 import org.apache.hadoop.fs.StorageStatistics;
 import org.apache.hadoop.fs.permission.FsPermission;
-import org.apache.hadoop.security.ProviderUtils;
+import org.apache.hadoop.fs.s3native.S3xLoginHelper;
 import org.apache.hadoop.util.Progressable;
 import org.apache.hadoop.util.VersionInfo;
 
+import static org.apache.commons.lang.StringUtils.*;
 import static org.apache.hadoop.fs.s3a.Constants.*;
 import static org.apache.hadoop.fs.s3a.S3AUtils.*;
 import static org.apache.hadoop.fs.s3a.Statistic.*;
@@ -140,7 +141,7 @@ public class S3AFileSystem extends FileSystem {
     try {
       instrumentation = new S3AInstrumentation(name);
 
-      uri = URI.create(name.getScheme() + "://" + name.getAuthority());
+      uri = S3xLoginHelper.buildFSURI(name);
       workingDir = new Path("/user", System.getProperty("user.name"))
           .makeQualified(this.uri, this.getWorkingDirectory());
 
@@ -399,53 +400,6 @@ public class S3AFileSystem extends FileSystem {
   }
 
   /**
-   * Return the access key and secret for S3 API use.
-   * Credentials may exist in configuration, within credential providers
-   * or indicated in the UserInfo of the name URI param.
-   * @param name the URI for which we need the access keys.
-   * @param conf the Configuration object to interogate for keys.
-   * @return AWSAccessKeys
-   */
-  AWSAccessKeys getAWSAccessKeys(URI name, Configuration conf)
-      throws IOException {
-    String accessKey = null;
-    String secretKey = null;
-    String userInfo = name.getUserInfo();
-    if (userInfo != null) {
-      int index = userInfo.indexOf(':');
-      if (index != -1) {
-        accessKey = userInfo.substring(0, index);
-        secretKey = userInfo.substring(index + 1);
-      } else {
-        accessKey = userInfo;
-      }
-    }
-    Configuration c = ProviderUtils.excludeIncompatibleCredentialProviders(
-          conf, S3AFileSystem.class);
-    if (accessKey == null) {
-      try {
-        final char[] key = c.getPassword(ACCESS_KEY);
-        if (key != null) {
-          accessKey = (new String(key)).trim();
-        }
-      } catch(IOException ioe) {
-        throw new IOException("Cannot find AWS access key.", ioe);
-      }
-    }
-    if (secretKey == null) {
-      try {
-        final char[] pass = c.getPassword(SECRET_KEY);
-        if (pass != null) {
-          secretKey = (new String(pass)).trim();
-        }
-      } catch(IOException ioe) {
-        throw new IOException("Cannot find AWS secret key.", ioe);
-      }
-    }
-    return new AWSAccessKeys(accessKey, secretKey);
-  }
-
-  /**
    * Create the standard credential provider, or load in one explicitly
    * identified in the configuration.
    * @param binding the S3 binding/bucket.
@@ -460,10 +414,10 @@ public class S3AFileSystem extends FileSystem {
 
     String className = conf.getTrimmed(AWS_CREDENTIALS_PROVIDER);
     if (StringUtils.isEmpty(className)) {
-      AWSAccessKeys creds = getAWSAccessKeys(binding, conf);
+      S3xLoginHelper.Login creds = getAWSAccessKeys(binding, conf);
       credentials = new AWSCredentialsProviderChain(
           new BasicAWSCredentialsProvider(
-              creds.getAccessKey(), creds.getAccessSecret()),
+              creds.getUser(), creds.getPassword()),
           new InstanceProfileCredentialsProvider(),
           new EnvironmentVariableCredentialsProvider());
 
@@ -551,10 +505,27 @@ public class S3AFileSystem extends FileSystem {
   }
 
   /**
-   * Opens an FSDataInputStream at the indicated Path.
-   * @param f the file name to open
-   * @param bufferSize the size of the buffer to be used.
+   * Check that a Path belongs to this FileSystem.
+   * Unlike the superclass, this version does not look at authority,
+   * only hostnames.
+   * @param path to check
+   * @throws IllegalArgumentException if there is an FS mismatch
    */
+  @Override
+  public void checkPath(Path path) {
+    S3xLoginHelper.checkPath(getConf(), getUri(), path, getDefaultPort());
+  }
+
+  @Override
+  protected URI canonicalizeUri(URI rawUri) {
+    return S3xLoginHelper.canonicalizeUri(rawUri, getDefaultPort());
+  }
+
+  /**
+     * Opens an FSDataInputStream at the indicated Path.
+     * @param f the file name to open
+     * @param bufferSize the size of the buffer to be used.
+     */
   public FSDataInputStream open(Path f, int bufferSize)
       throws IOException {
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4aefe119/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AUtils.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AUtils.java
index 062fca4..8033ac3 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AUtils.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AUtils.java
@@ -22,18 +22,26 @@ import com.amazonaws.AmazonClientException;
 import com.amazonaws.AmazonServiceException;
 import com.amazonaws.services.s3.model.AmazonS3Exception;
 import com.amazonaws.services.s3.model.S3ObjectSummary;
+import org.apache.commons.lang.StringUtils;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.s3native.S3xLoginHelper;
+import org.apache.hadoop.security.ProviderUtils;
 
 import java.io.EOFException;
 import java.io.FileNotFoundException;
 import java.io.IOException;
+import java.net.URI;
 import java.nio.file.AccessDeniedException;
 import java.util.Date;
 import java.util.Map;
 import java.util.concurrent.ExecutionException;
 
+import static org.apache.hadoop.fs.s3a.Constants.ACCESS_KEY;
+import static org.apache.hadoop.fs.s3a.Constants.SECRET_KEY;
+
 /**
  * Utility methods for S3A code.
  */
@@ -234,4 +242,42 @@ public final class S3AUtils {
 
     return date.getTime();
   }
+
+  /**
+   * Return the access key and secret for S3 API use.
+   * Credentials may exist in configuration, within credential providers
+   * or indicated in the UserInfo of the name URI param.
+   * @param name the URI for which we need the access keys.
+   * @param conf the Configuration object to interrogate for keys.
+   * @return AWSAccessKeys
+   * @throws IOException problems retrieving passwords from KMS.
+   */
+  public static S3xLoginHelper.Login getAWSAccessKeys(URI name,
+      Configuration conf) throws IOException {
+    S3xLoginHelper.Login login =
+        S3xLoginHelper.extractLoginDetailsWithWarnings(name);
+    Configuration c = ProviderUtils.excludeIncompatibleCredentialProviders(
+        conf, S3AFileSystem.class);
+    String accessKey = getPassword(c, ACCESS_KEY, login.getUser());
+    String secretKey = getPassword(c, SECRET_KEY, login.getPassword());
+    return new S3xLoginHelper.Login(accessKey, secretKey);
+  }
+
+  private static String getPassword(Configuration conf, String key, String val)
+      throws IOException {
+    if (StringUtils.isEmpty(val)) {
+      try {
+        final char[] pass = conf.getPassword(key);
+        if (pass != null) {
+          return (new String(pass)).trim();
+        } else {
+          return "";
+        }
+      } catch (IOException ioe) {
+        throw new IOException("Cannot find password option " + key, ioe);
+      }
+    } else {
+      return val;
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4aefe119/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3native/NativeS3FileSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3native/NativeS3FileSystem.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3native/NativeS3FileSystem.java
index 0ad8e5f..d646726 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3native/NativeS3FileSystem.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3native/NativeS3FileSystem.java
@@ -333,7 +333,7 @@ public class NativeS3FileSystem extends FileSystem {
     }
     store.initialize(uri, conf);
     setConf(conf);
-    this.uri = URI.create(uri.getScheme() + "://" + uri.getAuthority());
+    this.uri = S3xLoginHelper.buildFSURI(uri);
     this.workingDir =
       new Path("/user", System.getProperty("user.name")).makeQualified(this.uri, this.getWorkingDirectory());
   }
@@ -388,6 +388,23 @@ public class NativeS3FileSystem extends FileSystem {
     return new Path(workingDir, path);
   }
 
+  /**
+   * Check that a Path belongs to this FileSystem.
+   * Unlike the superclass, this version does not look at authority,
+   * only hostnames.
+   * @param path to check
+   * @throws IllegalArgumentException if there is an FS mismatch
+   */
+  @Override
+  protected void checkPath(Path path) {
+    S3xLoginHelper.checkPath(getConf(), getUri(), path, getDefaultPort());
+  }
+
+  @Override
+  protected URI canonicalizeUri(URI rawUri) {
+    return S3xLoginHelper.canonicalizeUri(rawUri, getDefaultPort());
+  }
+
   /** This optional operation is not yet supported. */
   @Override
   public FSDataOutputStream append(Path f, int bufferSize,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4aefe119/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3native/S3xLoginHelper.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3native/S3xLoginHelper.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3native/S3xLoginHelper.java
new file mode 100644
index 0000000..bc8c2e6
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3native/S3xLoginHelper.java
@@ -0,0 +1,283 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.s3native;
+
+import org.apache.commons.lang.StringUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.UnsupportedEncodingException;
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.net.URLDecoder;
+import java.util.Objects;
+
+import static org.apache.commons.lang.StringUtils.equalsIgnoreCase;
+
+/**
+ * Class to aid logging in to S3 endpoints.
+ * It is in S3N so that it can be used across all S3 filesystems.
+ */
+public final class S3xLoginHelper {
+  private static final Logger LOG =
+      LoggerFactory.getLogger(S3xLoginHelper.class);
+
+  private S3xLoginHelper() {
+  }
+
+  public static final String LOGIN_WARNING =
+      "The Filesystem URI contains login details."
+      +" This is insecure and may be unsupported in future.";
+
+  /**
+   * Build the filesystem URI. This can include stripping down of part
+   * of the URI.
+   * @param uri filesystem uri
+   * @return the URI to use as the basis for FS operation and qualifying paths.
+   * @throws IllegalArgumentException if the URI is in some way invalid.
+   */
+  public static URI buildFSURI(URI uri) {
+    Objects.requireNonNull(uri, "null uri");
+    Objects.requireNonNull(uri.getScheme(), "null uri.getScheme()");
+    if (uri.getHost() == null && uri.getAuthority() != null) {
+      Objects.requireNonNull(uri.getHost(), "null uri host." +
+          " This can be caused by unencoded / in the password string");
+    }
+    Objects.requireNonNull(uri.getHost(), "null uri host.");
+    return URI.create(uri.getScheme() + "://" + uri.getHost());
+  }
+
+  /**
+   * Create a stripped down string value for error messages.
+   * @param pathUri URI
+   * @return a shortened schema://host/path value
+   */
+  public static String toString(URI pathUri) {
+    return pathUri != null
+        ? String.format("%s://%s/%s",
+        pathUri.getScheme(), pathUri.getHost(), pathUri.getPath())
+        : "(null URI)";
+  }
+
+  /**
+   * Extract the login details from a URI, logging a warning if
+   * the URI contains these.
+   * @param name URI of the filesystem
+   * @return a login tuple, possibly empty.
+   */
+  public static Login extractLoginDetailsWithWarnings(URI name) {
+    Login login = extractLoginDetails(name);
+    if (login.hasLogin()) {
+      LOG.warn(LOGIN_WARNING);
+    }
+    return login;
+  }
+
+  /**
+   * Extract the login details from a URI.
+   * @param name URI of the filesystem
+   * @return a login tuple, possibly empty.
+   */
+  public static Login extractLoginDetails(URI name) {
+    try {
+      String authority = name.getAuthority();
+      if (authority == null) {
+        return Login.EMPTY;
+      }
+      int loginIndex = authority.indexOf('@');
+      if (loginIndex < 0) {
+        // no login
+        return Login.EMPTY;
+      }
+      String login = authority.substring(0, loginIndex);
+      int loginSplit = login.indexOf(':');
+      if (loginSplit > 0) {
+        String user = login.substring(0, loginSplit);
+        String password = URLDecoder.decode(login.substring(loginSplit + 1),
+            "UTF-8");
+        return new Login(user, password);
+      } else if (loginSplit == 0) {
+        // there is no user, just a password. In this case, there's no login
+        return Login.EMPTY;
+      } else {
+        return new Login(login, "");
+      }
+    } catch (UnsupportedEncodingException e) {
+      // this should never happen; translate it if it does.
+      throw new RuntimeException(e);
+    }
+  }
+
+  /**
+   * Canonicalize the given URI.
+   *
+   * This strips out login information.
+   *
+   * @return a new, canonicalized URI.
+   */
+  public static URI canonicalizeUri(URI uri, int defaultPort) {
+    if (uri.getPort() == -1 && defaultPort > 0) {
+      // reconstruct the uri with the default port set
+      try {
+        uri = new URI(uri.getScheme(),
+            null,
+            uri.getHost(),
+            defaultPort,
+            uri.getPath(),
+            uri.getQuery(),
+            uri.getFragment());
+      } catch (URISyntaxException e) {
+        // Should never happen!
+        throw new AssertionError("Valid URI became unparseable: " +
+            uri);
+      }
+    }
+
+    return uri;
+  }
+
+  /**
+   * Check the path, ignoring authentication details.
+   * See {@link FileSystem#checkPath(Path)} for the operation of this.
+   *
+   * Essentially
+   * <ol>
+   *   <li>The URI is canonicalized.</li>
+   *   <li>If the schemas match, the hosts are compared.</li>
+   *   <li>If there is a mismatch between null/non-null host, the default FS
+   *   values are used to patch in the host.</li>
+   * </ol>
+   * That all originates in the core FS; the sole change here being to use
+   * {@link URI#getHost()} over {@link URI#getAuthority()}. Some of that
+   * code looks a relic of the code anti-pattern of using "hdfs:file.txt"
+   * to define the path without declaring the hostname. It's retained
+   * for compatibility.
+   * @param conf FS configuration
+   * @param fsUri the FS URI
+   * @param path path to check
+   * @param defaultPort default port of FS
+   */
+  public static void checkPath(Configuration conf,
+      URI fsUri,
+      Path path,
+      int defaultPort) {
+    URI pathUri = path.toUri();
+    String thatScheme = pathUri.getScheme();
+    if (thatScheme == null) {
+      // fs is relative
+      return;
+    }
+    URI thisUri = canonicalizeUri(fsUri, defaultPort);
+    String thisScheme = thisUri.getScheme();
+    //hostname and scheme are not case sensitive in these checks
+    if (equalsIgnoreCase(thisScheme, thatScheme)) {// schemes match
+      String thisHost = thisUri.getHost();
+      String thatHost = pathUri.getHost();
+      if (thatHost == null &&                // path's host is null
+          thisHost != null) {                // fs has a host
+        URI defaultUri = FileSystem.getDefaultUri(conf);
+        if (equalsIgnoreCase(thisScheme, defaultUri.getScheme())) {
+          pathUri = defaultUri; // schemes match, so use this uri instead
+        } else {
+          pathUri = null; // can't determine auth of the path
+        }
+      }
+      if (pathUri != null) {
+        // canonicalize uri before comparing with this fs
+        pathUri = canonicalizeUri(pathUri, defaultPort);
+        thatHost = pathUri.getHost();
+        if (thisHost == thatHost ||       // hosts match
+            (thisHost != null &&
+                 equalsIgnoreCase(thisHost, thatHost))) {
+          return;
+        }
+      }
+    }
+    // make sure the exception strips out any auth details
+    throw new IllegalArgumentException(
+        "Wrong FS " + S3xLoginHelper.toString(pathUri)
+            + " -expected " + fsUri);
+  }
+
+  /**
+   * Simple tuple of login details.
+   */
+  public static class Login {
+    private final String user;
+    private final String password;
+
+    public static final Login EMPTY = new Login();
+
+    /**
+     * Create an instance with no login details.
+     * Calls to {@link #hasLogin()} return false.
+     */
+    public Login() {
+      this("", "");
+    }
+
+    public Login(String user, String password) {
+      this.user = user;
+      this.password = password;
+    }
+
+    /**
+     * Predicate to verify login details are defined.
+     * @return true if the username is defined (not null, not empty).
+     */
+    public boolean hasLogin() {
+      return StringUtils.isNotEmpty(user);
+    }
+
+    /**
+     * Equality test matches user and password.
+     * @param o other object
+     * @return true if the objects are considered equivalent.
+     */
+    @Override
+    public boolean equals(Object o) {
+      if (this == o) {
+        return true;
+      }
+      if (o == null || getClass() != o.getClass()) {
+        return false;
+      }
+      Login that = (Login) o;
+      return Objects.equals(user, that.user) &&
+          Objects.equals(password, that.password);
+    }
+
+    @Override
+    public int hashCode() {
+      return Objects.hash(user, password);
+    }
+
+    public String getUser() {
+      return user;
+    }
+
+    public String getPassword() {
+      return password;
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4aefe119/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 606275c..79c9349 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
@@ -69,8 +69,11 @@ access to the data. Anyone with the credentials can not only read your datasets
 \u2014they can delete them.
 
 Do not inadvertently share these credentials through means such as
-1. Checking in Hadoop configuration files containing the credentials.
+1. Checking in to SCM any configuration files containing the secrets.
 1. Logging them to a console, as they invariably end up being seen.
+1. Defining filesystem URIs with the credentials in the URL, such as
+`s3a://AK0010:secret@landsat/`. They will end up in logs and error messages.
+1. Including the secrets in bug reports.
 
 If you do any of these: change your credentials immediately!
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4aefe119/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3/TestS3FileSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3/TestS3FileSystem.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3/TestS3FileSystem.java
index f21989c..4947f08 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3/TestS3FileSystem.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3/TestS3FileSystem.java
@@ -27,24 +27,26 @@ import org.apache.hadoop.conf.Configuration;
 
 public class TestS3FileSystem extends TestCase {
 
+  public static final URI EXPECTED = URI.create("s3://c");
+
   public void testInitialization() throws IOException {
-    initializationTest("s3://a:b@c", "s3://a:b@c");
-    initializationTest("s3://a:b@c/", "s3://a:b@c");
-    initializationTest("s3://a:b@c/path", "s3://a:b@c");
-    initializationTest("s3://a@c", "s3://a@c");
-    initializationTest("s3://a@c/", "s3://a@c");
-    initializationTest("s3://a@c/path", "s3://a@c");
-    initializationTest("s3://c", "s3://c");
-    initializationTest("s3://c/", "s3://c");
-    initializationTest("s3://c/path", "s3://c");
+    initializationTest("s3://a:b@c");
+    initializationTest("s3://a:b@c/");
+    initializationTest("s3://a:b@c/path");
+    initializationTest("s3://a@c");
+    initializationTest("s3://a@c/");
+    initializationTest("s3://a@c/path");
+    initializationTest("s3://c");
+    initializationTest("s3://c/");
+    initializationTest("s3://c/path");
   }
   
-  private void initializationTest(String initializationUri, String expectedUri)
+  private void initializationTest(String initializationUri)
     throws IOException {
     
     S3FileSystem fs = new S3FileSystem(new InMemoryFileSystemStore());
     fs.initialize(URI.create(initializationUri), new Configuration());
-    assertEquals(URI.create(expectedUri), fs.getUri());
+    assertEquals(EXPECTED, fs.getUri());
   }
 
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4aefe119/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AConfiguration.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AConfiguration.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AConfiguration.java
index 513cae2..29bfd59 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AConfiguration.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AConfiguration.java
@@ -27,6 +27,7 @@ import org.apache.commons.lang.reflect.FieldUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.contract.ContractTestUtils;
+import org.apache.hadoop.fs.s3native.S3xLoginHelper;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.rules.Timeout;
@@ -210,12 +211,11 @@ public class TestS3AConfiguration {
 
     provisionAccessKeys(conf);
 
-    S3AFileSystem s3afs = new S3AFileSystem();
     conf.set(Constants.ACCESS_KEY, EXAMPLE_ID + "LJM");
-    S3AFileSystem.AWSAccessKeys creds =
-        s3afs.getAWSAccessKeys(new URI("s3a://foobar"), conf);
-    assertEquals("AccessKey incorrect.", EXAMPLE_ID, creds.getAccessKey());
-    assertEquals("SecretKey incorrect.", EXAMPLE_KEY, creds.getAccessSecret());
+    S3xLoginHelper.Login creds =
+        S3AUtils.getAWSAccessKeys(new URI("s3a://foobar"), conf);
+    assertEquals("AccessKey incorrect.", EXAMPLE_ID, creds.getUser());
+    assertEquals("SecretKey incorrect.", EXAMPLE_KEY, creds.getPassword());
   }
 
   void provisionAccessKeys(final Configuration conf) throws Exception {
@@ -241,13 +241,12 @@ public class TestS3AConfiguration {
 
     provisionAccessKeys(conf);
 
-    S3AFileSystem s3afs = new S3AFileSystem();
     conf.set(Constants.ACCESS_KEY, EXAMPLE_ID + "LJM");
     URI uriWithUserInfo = new URI("s3a://123:456@foobar");
-    S3AFileSystem.AWSAccessKeys creds =
-        s3afs.getAWSAccessKeys(uriWithUserInfo, conf);
-    assertEquals("AccessKey incorrect.", "123", creds.getAccessKey());
-    assertEquals("SecretKey incorrect.", "456", creds.getAccessSecret());
+    S3xLoginHelper.Login creds =
+        S3AUtils.getAWSAccessKeys(uriWithUserInfo, conf);
+    assertEquals("AccessKey incorrect.", "123", creds.getUser());
+    assertEquals("SecretKey incorrect.", "456", creds.getPassword());
   }
 
   @Test
@@ -263,13 +262,12 @@ public class TestS3AConfiguration {
 
     provisionAccessKeys(conf);
 
-    S3AFileSystem s3afs = new S3AFileSystem();
     conf.set(Constants.ACCESS_KEY, EXAMPLE_ID + "LJM");
     URI uriWithUserInfo = new URI("s3a://123@foobar");
-    S3AFileSystem.AWSAccessKeys creds =
-        s3afs.getAWSAccessKeys(uriWithUserInfo, conf);
-    assertEquals("AccessKey incorrect.", "123", creds.getAccessKey());
-    assertEquals("SecretKey incorrect.", EXAMPLE_KEY, creds.getAccessSecret());
+    S3xLoginHelper.Login creds =
+        S3AUtils.getAWSAccessKeys(uriWithUserInfo, conf);
+    assertEquals("AccessKey incorrect.", "123", creds.getUser());
+    assertEquals("SecretKey incorrect.", EXAMPLE_KEY, creds.getPassword());
   }
 
   @Test
@@ -289,12 +287,11 @@ public class TestS3AConfiguration {
         EXAMPLE_KEY.toCharArray());
     provider.flush();
 
-    S3AFileSystem s3afs = new S3AFileSystem();
     conf.set(Constants.ACCESS_KEY, EXAMPLE_ID);
-    S3AFileSystem.AWSAccessKeys creds =
-        s3afs.getAWSAccessKeys(new URI("s3a://foobar"), conf);
-    assertEquals("AccessKey incorrect.", EXAMPLE_ID, creds.getAccessKey());
-    assertEquals("SecretKey incorrect.", EXAMPLE_KEY, creds.getAccessSecret());
+    S3xLoginHelper.Login creds =
+        S3AUtils.getAWSAccessKeys(new URI("s3a://foobar"), conf);
+    assertEquals("AccessKey incorrect.", EXAMPLE_ID, creds.getUser());
+    assertEquals("SecretKey incorrect.", EXAMPLE_KEY, creds.getPassword());
   }
 
   @Test
@@ -314,12 +311,11 @@ public class TestS3AConfiguration {
         EXAMPLE_ID.toCharArray());
     provider.flush();
 
-    S3AFileSystem s3afs = new S3AFileSystem();
     conf.set(Constants.SECRET_KEY, EXAMPLE_KEY);
-    S3AFileSystem.AWSAccessKeys creds =
-        s3afs.getAWSAccessKeys(new URI("s3a://foobar"), conf);
-    assertEquals("AccessKey incorrect.", EXAMPLE_ID, creds.getAccessKey());
-    assertEquals("SecretKey incorrect.", EXAMPLE_KEY, creds.getAccessSecret());
+    S3xLoginHelper.Login creds =
+        S3AUtils.getAWSAccessKeys(new URI("s3a://foobar"), conf);
+    assertEquals("AccessKey incorrect.", EXAMPLE_ID, creds.getUser());
+    assertEquals("SecretKey incorrect.", EXAMPLE_KEY, creds.getPassword());
   }
 
   @Test
@@ -345,13 +341,12 @@ public class TestS3AConfiguration {
     // using the original config with the s3a provider in the path.
     provisionAccessKeys(c);
 
-    S3AFileSystem s3afs = new S3AFileSystem();
     conf.set(Constants.ACCESS_KEY, EXAMPLE_ID + "LJM");
     URI uriWithUserInfo = new URI("s3a://123:456@foobar");
-    S3AFileSystem.AWSAccessKeys creds =
-        s3afs.getAWSAccessKeys(uriWithUserInfo, conf);
-    assertEquals("AccessKey incorrect.", "123", creds.getAccessKey());
-    assertEquals("SecretKey incorrect.", "456", creds.getAccessSecret());
+    S3xLoginHelper.Login creds =
+        S3AUtils.getAWSAccessKeys(uriWithUserInfo, conf);
+    assertEquals("AccessKey incorrect.", "123", creds.getUser());
+    assertEquals("SecretKey incorrect.", "456", creds.getPassword());
 
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4aefe119/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3ACredentialsInURL.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3ACredentialsInURL.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3ACredentialsInURL.java
new file mode 100644
index 0000000..f1da72b
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3ACredentialsInURL.java
@@ -0,0 +1,153 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.s3a;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.IOUtils;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Assume;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.Timeout;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.io.UnsupportedEncodingException;
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.net.URLEncoder;
+import java.nio.file.AccessDeniedException;
+
+import static org.apache.hadoop.fs.s3a.S3ATestConstants.TEST_FS_S3A_NAME;
+
+/**
+ * Tests that credentials can go into the URL. This includes a valid
+ * set, and a check that an invalid set do at least get stripped out
+ * of the final URI
+ */
+public class TestS3ACredentialsInURL extends Assert {
+  private S3AFileSystem fs;
+  private static final Logger LOG =
+      LoggerFactory.getLogger(TestS3ACredentialsInURL.class);
+  @Rule
+  public Timeout testTimeout = new Timeout(30 * 60 * 1000);
+
+  @After
+  public void teardown() {
+    IOUtils.closeStream(fs);
+  }
+
+  /**
+   * Test instantiation.
+   * @throws Throwable
+   */
+  @Test
+  public void testInstantiateFromURL() throws Throwable {
+
+    Configuration conf = new Configuration();
+    String accessKey = conf.get(Constants.ACCESS_KEY);
+    String secretKey = conf.get(Constants.SECRET_KEY);
+    String fsname = conf.getTrimmed(TEST_FS_S3A_NAME, "");
+    Assume.assumeNotNull(fsname, accessKey, secretKey);
+    URI original = new URI(fsname);
+    URI secretsURI = createUriWithEmbeddedSecrets(original,
+        accessKey, secretKey);
+    if (secretKey.contains("/")) {
+      assertTrue("test URI encodes the / symbol", secretsURI.toString().
+          contains("%2F"));
+    }
+    assertFalse("Does not contain secrets", original.equals(secretsURI));
+
+    conf.set(TEST_FS_S3A_NAME, secretsURI.toString());
+    conf.unset(Constants.ACCESS_KEY);
+    conf.unset(Constants.SECRET_KEY);
+    fs = S3ATestUtils.createTestFileSystem(conf);
+    String fsURI = fs.getUri().toString();
+    assertFalse("FS URI contains a @ symbol", fsURI.contains("@"));
+    assertFalse("FS URI contains a % symbol", fsURI.contains("%"));
+    if (!original.toString().startsWith(fsURI)) {
+      fail("Filesystem URI does not match original");
+    }
+    validate("original path", new Path(original));
+    validate("bare path", new Path("/"));
+    validate("secrets path", new Path(secretsURI));
+  }
+
+  private void validate(String text, Path path) throws IOException {
+    try {
+      fs.canonicalizeUri(path.toUri());
+      fs.checkPath(path);
+      assertTrue(text + " Not a directory",
+          fs.getFileStatus(new Path("/")).isDirectory());
+      fs.globStatus(path);
+    } catch (AssertionError e) {
+      throw e;
+    } catch (Exception e) {
+      LOG.debug("{} failure: {}", text, e, e);
+      fail(text + " Test failed");
+    }
+  }
+  
+  /**
+   * Set up some invalid credentials, verify login is rejected.
+   * @throws Throwable
+   */
+  @Test
+  public void testInvalidCredentialsFail() throws Throwable {
+    Configuration conf = new Configuration();
+    String fsname = conf.getTrimmed(TEST_FS_S3A_NAME, "");
+    Assume.assumeNotNull(fsname);
+    URI original = new URI(fsname);
+    URI testURI = createUriWithEmbeddedSecrets(original, "user", "//");
+
+    conf.set(TEST_FS_S3A_NAME, testURI.toString());
+    fs = S3ATestUtils.createTestFileSystem(conf);
+    try {
+      S3AFileStatus status = fs.getFileStatus(new Path("/"));
+      fail("Expected an AccessDeniedException, got " + status);
+    } catch (AccessDeniedException e) {
+      // expected
+    }
+
+  }
+
+  private URI createUriWithEmbeddedSecrets(URI original,
+      String accessKey,
+      String secretKey) throws URISyntaxException,
+      UnsupportedEncodingException {
+    String encodedSecretKey = URLEncoder.encode(secretKey, "UTF-8");
+    String formattedString = String.format("%s://%s:%s@%s/%s/",
+        original.getScheme(),
+        accessKey,
+        encodedSecretKey,
+        original.getHost(),
+        original.getPath());
+    URI testURI;
+    try {
+      testURI = new URI(formattedString);
+    } catch (URISyntaxException e) {
+      // inner cause is stripped to keep any secrets out of stack traces
+      throw new URISyntaxException("", "Could not encode URI");
+    }
+    return testURI;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4aefe119/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3native/TestS3xLoginHelper.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3native/TestS3xLoginHelper.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3native/TestS3xLoginHelper.java
new file mode 100644
index 0000000..bd2ac1e
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3native/TestS3xLoginHelper.java
@@ -0,0 +1,197 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.s3native;
+
+import org.apache.hadoop.fs.Path;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.net.URI;
+import java.net.URISyntaxException;
+
+/**
+ * Test how URIs and login details are extracted from URIs.
+ */
+public class TestS3xLoginHelper extends Assert {
+  public static final String BUCKET = "s3a://bucket";
+  private static final URI ENDPOINT = uri(BUCKET);
+  public static final String S = "%2f";
+  public static final String USER = "user";
+  public static final String PASS = "pass";
+  public static final String PASLASHSLASH = "pa" + S + S;
+
+  public static final URI WITH_USER_AND_PASS = uri("s3a://user:pass@bucket");
+  public static final Path PATH_WITH_LOGIN =
+      new Path(uri("s3a://user:pass@bucket/dest"));
+
+  public static final URI WITH_SLASH_IN_PASS = uri(
+      "s3a://user:" + PASLASHSLASH + "@bucket");
+  public static final URI USER_NO_PASS = uri("s3a://user@bucket");
+  public static final URI WITH_USER_AND_COLON = uri("s3a://user:@bucket");
+  public static final URI NO_USER = uri("s3a://:pass@bucket");
+  public static final URI NO_USER_NO_PASS = uri("s3a://:@bucket");
+  public static final URI NO_USER_NO_PASS_TWO_COLON = uri("s3a://::@bucket");
+
+  /**
+   * Construct a URI; raises an RTE if it won't parse.
+   * This allows it to be used in static constructors.
+   * @param s URI string
+   * @return the URI
+   * @throws RuntimeException on a URI syntax problem
+   */
+  private static URI uri(String s) {
+    try {
+      return new URI(s);
+    } catch (URISyntaxException e) {
+      throw new RuntimeException(e.toString(), e);
+    }
+  }
+
+  /**
+   * Assert that a built up FS URI matches the endpoint.
+   * @param uri URI to build the FS UIR from
+   */
+  private void assertMatchesEndpoint(URI uri) {
+    assertEquals("Source " + uri,
+        ENDPOINT, S3xLoginHelper.buildFSURI(uri));
+  }
+
+  /**
+   * Assert that the login/pass details from a URI match that expected.
+   * @param user username
+   * @param pass password
+   * @param uri URI to build login details from
+   * @return the login tuple
+   */
+  private S3xLoginHelper.Login assertMatchesLogin(String user,
+      String pass, URI uri) {
+    S3xLoginHelper.Login expected = new S3xLoginHelper.Login(user,
+        pass);
+    S3xLoginHelper.Login actual = S3xLoginHelper.extractLoginDetails(
+        uri);
+    if (!expected.equals(actual)) {
+      Assert.fail("Source " + uri
+          + " login expected=:" + toString(expected)
+          + " actual=" + toString(actual));
+    }
+    return actual;
+  }
+
+  @Test
+  public void testSimpleFSURI() throws Throwable {
+    assertMatchesEndpoint(ENDPOINT);
+  }
+
+  @Test
+  public void testLoginSimple() throws Throwable {
+    S3xLoginHelper.Login login = assertMatchesLogin("", "", ENDPOINT);
+    assertFalse("Login of " + login, login.hasLogin());
+  }
+
+  @Test
+  public void testLoginWithUserAndPass() throws Throwable {
+    S3xLoginHelper.Login login = assertMatchesLogin(USER, PASS,
+        WITH_USER_AND_PASS);
+    assertTrue("Login of " + login, login.hasLogin());
+  }
+
+  @Test
+  public void testLoginWithSlashInPass() throws Throwable {
+    assertMatchesLogin(USER, "pa//", WITH_SLASH_IN_PASS);
+  }
+
+  @Test
+  public void testLoginWithUser() throws Throwable {
+    assertMatchesLogin(USER, "", USER_NO_PASS);
+  }
+
+  @Test
+  public void testLoginWithUserAndColon() throws Throwable {
+    assertMatchesLogin(USER, "", WITH_USER_AND_COLON);
+  }
+
+  @Test
+  public void testLoginNoUser() throws Throwable {
+    assertMatchesLogin("", "", NO_USER);
+  }
+
+  @Test
+  public void testLoginNoUserNoPass() throws Throwable {
+    assertMatchesLogin("", "", NO_USER_NO_PASS);
+  }
+
+  @Test
+  public void testLoginNoUserNoPassTwoColon() throws Throwable {
+    assertMatchesLogin("", "", NO_USER_NO_PASS_TWO_COLON);
+  }
+
+  @Test
+  public void testFsUriWithUserAndPass() throws Throwable {
+    assertMatchesEndpoint(WITH_USER_AND_PASS);
+  }
+
+  @Test
+  public void testFsUriWithSlashInPass() throws Throwable {
+    assertMatchesEndpoint(WITH_SLASH_IN_PASS);
+  }
+
+  @Test
+  public void testFsUriWithUser() throws Throwable {
+    assertMatchesEndpoint(USER_NO_PASS);
+  }
+
+  @Test
+  public void testFsUriWithUserAndColon() throws Throwable {
+    assertMatchesEndpoint(WITH_USER_AND_COLON);
+  }
+
+  @Test
+  public void testFsiNoUser() throws Throwable {
+    assertMatchesEndpoint(NO_USER);
+  }
+
+  @Test
+  public void testFsUriNoUserNoPass() throws Throwable {
+    assertMatchesEndpoint(NO_USER_NO_PASS);
+  }
+
+  @Test
+  public void testFsUriNoUserNoPassTwoColon() throws Throwable {
+    assertMatchesEndpoint(NO_USER_NO_PASS_TWO_COLON);
+  }
+
+  @Test
+  public void testPathURIFixup() throws Throwable {
+
+  }
+
+
+  /**
+   * Stringifier. Kept in the code to avoid accidental logging in production
+   * code.
+   * @return login details for assertions.
+   */
+  public String toString(S3xLoginHelper.Login login) {
+    final StringBuilder sb = new StringBuilder("LoginTuple{");
+    sb.append("<'").append(login.getUser()).append('\'');
+    sb.append(", '").append(login.getPassword()).append('\'');
+    sb.append('>');
+    return sb.toString();
+  }
+}


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


[37/39] hadoop git commit: Revert "HADOOP-12718. Incorrect error message by fs -put local dir without permission. (John Zhuge via Yongjun Zhang)"

Posted by ae...@apache.org.
Revert "HADOOP-12718. Incorrect error message by fs -put local dir without permission. (John Zhuge via Yongjun Zhang)"

This reverts commit 97056c3355810a803f07baca89b89e2bf6bb7201.

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


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

Branch: refs/heads/HDFS-1312
Commit: 5f6bc65bb31270f2b5dfdfd941a0568fc1f3337f
Parents: 20f2799
Author: Chris Nauroth <cn...@apache.org>
Authored: Fri Jun 17 23:50:22 2016 -0700
Committer: Chris Nauroth <cn...@apache.org>
Committed: Fri Jun 17 23:50:22 2016 -0700

----------------------------------------------------------------------
 .../src/main/java/org/apache/hadoop/fs/RawLocalFileSystem.java  | 5 -----
 1 file changed, 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/5f6bc65b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/RawLocalFileSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/RawLocalFileSystem.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/RawLocalFileSystem.java
index 8ef8392..cc41f4a 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/RawLocalFileSystem.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/RawLocalFileSystem.java
@@ -33,7 +33,6 @@ import java.io.OutputStream;
 import java.io.FileDescriptor;
 import java.net.URI;
 import java.nio.ByteBuffer;
-import java.nio.file.AccessDeniedException;
 import java.nio.file.Files;
 import java.nio.file.NoSuchFileException;
 import java.nio.file.attribute.BasicFileAttributes;
@@ -472,10 +471,6 @@ public class RawLocalFileSystem extends FileSystem {
     if (localf.isDirectory()) {
       String[] names = localf.list();
       if (names == null) {
-        if (!localf.canRead()) {
-          throw new AccessDeniedException("cannot open directory " + f +
-              ": Permission denied");
-        }
         return null;
       }
       results = new FileStatus[names.length];


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


[13/39] hadoop git commit: YARN-5223. Container line in yarn logs output for a live application should include the hostname for the container. Contributed by Xuan Gong.

Posted by ae...@apache.org.
YARN-5223. Container line in yarn logs output for a live application should include the hostname for the container. Contributed by Xuan Gong.


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

Branch: refs/heads/HDFS-1312
Commit: 25064fb2fb79751cedbb8019900d811e07549ecf
Parents: d628442
Author: Varun Vasudev <vv...@apache.org>
Authored: Wed Jun 15 12:42:44 2016 +0530
Committer: Varun Vasudev <vv...@apache.org>
Committed: Wed Jun 15 12:42:44 2016 +0530

----------------------------------------------------------------------
 .../main/java/org/apache/hadoop/yarn/client/cli/LogsCLI.java    | 3 ++-
 .../java/org/apache/hadoop/yarn/client/cli/TestLogsCLI.java     | 2 ++
 .../org/apache/hadoop/yarn/logaggregation/LogCLIHelpers.java    | 5 +++--
 3 files changed, 7 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/25064fb2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/LogsCLI.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/LogsCLI.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/LogsCLI.java
index c909402..ebd5428 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/LogsCLI.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/LogsCLI.java
@@ -393,7 +393,8 @@ public class LogsCLI extends Configured implements Tool {
       newOptions.setLogTypes(matchedFiles);
 
       Client webServiceClient = Client.create();
-      String containerString = "\n\nContainer: " + containerIdStr;
+      String containerString = "\n\nContainer: " + containerIdStr + " on "
+          + nodeId;
       out.println(containerString);
       out.println(StringUtils.repeat("=", containerString.length()));
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/25064fb2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestLogsCLI.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestLogsCLI.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestLogsCLI.java
index db40b50..10047e3 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestLogsCLI.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestLogsCLI.java
@@ -463,6 +463,8 @@ public class TestLogsCLI {
         "Hello container_0_0001_01_000003 in syslog!"));
     assertTrue(sysOutStream.toString().contains(
         "Hello container_0_0001_01_000003 in stdout!"));
+    assertTrue(sysOutStream.toString().contains(
+        containerId3 + " on " + LogAggregationUtils.getNodeString(nodeId)));
     sysOutStream.reset();
 
     fs.delete(new Path(remoteLogRootDir), true);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/25064fb2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/LogCLIHelpers.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/LogCLIHelpers.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/LogCLIHelpers.java
index 26b2b01..e2e3b34 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/LogCLIHelpers.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/LogCLIHelpers.java
@@ -153,7 +153,8 @@ public class LogCLIHelpers implements Configurable {
         AggregatedLogFormat.LogReader reader = null;
         PrintStream out = createPrintStream(localDir, fileName, containerId);
         try {
-          String containerString = "\n\nContainer: " + containerId;
+          String containerString = "\n\nContainer: " + containerId + " on "
+              + thisNodeFile.getPath().getName();
           out.println(containerString);
           out.println(StringUtils.repeat("=", containerString.length()));
           reader =
@@ -220,7 +221,7 @@ public class LogCLIHelpers implements Configurable {
               thisNodeFile.getPath());
           out = createPrintStream(localDir, thisNodeFile.getPath().getName(),
               containerId);
-          out.println(containerId);
+          out.println(containerId + " on " + thisNodeFile.getPath().getName());
           out.println(StringUtils.repeat("=", containerId.length()));
           if (logType == null || logType.isEmpty()) {
             if (dumpAContainerLogs(containerId, reader, out,


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


[33/39] hadoop git commit: HDFS-10493. Add links to datanode web UI in namenode datanodes page. Contributed by Weiwei Yang.

Posted by ae...@apache.org.
HDFS-10493. Add links to datanode web UI in namenode datanodes page. Contributed by Weiwei Yang.


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

Branch: refs/heads/HDFS-1312
Commit: 280069510b71d81a150b2b28d9fa987891d82774
Parents: c35fa4a
Author: Kihwal Lee <ki...@apache.org>
Authored: Fri Jun 17 11:05:44 2016 -0500
Committer: Kihwal Lee <ki...@apache.org>
Committed: Fri Jun 17 11:05:44 2016 -0500

----------------------------------------------------------------------
 .../hadoop-hdfs/src/main/webapps/hdfs/dfshealth.html      |  9 +++++++++
 .../hadoop-hdfs/src/main/webapps/hdfs/dfshealth.js        | 10 ++++++++++
 2 files changed, 19 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/28006951/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/dfshealth.html
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/dfshealth.html b/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/dfshealth.html
index 66c0dec..4fa2e4c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/dfshealth.html
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/dfshealth.html
@@ -300,6 +300,7 @@
   <thead>
     <tr>
       <th>Node</th>
+      <th>Http Address</th>
       <th>Last contact</th>
       <th style="width:180px; text-align:center">Capacity</th>
       <th>Blocks</th>
@@ -310,6 +311,13 @@
   {#LiveNodes}
   <tr>
     <td ng-value="{state}-{name}" class="dfshealth-node-icon dfshealth-node-{state}">{name} ({xferaddr})</td>
+    <td ng-value="{state}-{name}">
+    {@select key=secureMode}
+      {@eq value="off"}<a href='//{infoAddr}'>{infoAddr}</a>{/eq}
+      {@eq value="on"}<a href='//{infoSecureAddr}'>{infoSecureAddr}</a>{/eq}
+      {@default}<a href='//{infoAddr}'>{infoAddr}</a>{/default}
+    {/select}
+    </td>
     <td ng-value="{lastContact}">{lastContact}s</td>
     <td ng-value="{usedPercentage}">
       <div>
@@ -328,6 +336,7 @@
   {#DeadNodes}
   <tr class="danger">
     <td ng-value="{state}-{name}" class="dfshealth-node-icon dfshealth-node-{state}">{name} ({xferaddr})</td>
+    <td></td>
     <td>{#helper_relative_time value="{lastContact}"/}</td>
     <td></td>
     <td></td>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/28006951/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/dfshealth.js
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/dfshealth.js b/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/dfshealth.js
index e7245df..e2908c0 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/dfshealth.js
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/dfshealth.js
@@ -210,6 +210,15 @@
         for (var i = 0, e = nodes.length; i < e; ++i) {
           var n = nodes[i];
           n.usedPercentage = Math.round((n.used + n.nonDfsUsedSpace) * 1.0 / n.capacity * 100);
+
+          var addr = n.infoSecureAddr;
+          var position = addr.lastIndexOf(":");
+          var port = addr.substring(position + 1, addr.length);
+          n.secureMode = "off";
+          if (port != 0) {
+            n.secureMode = "on";
+          }
+
           if (n.adminState === "In Service") {
             n.state = "alive";
           } else if (nodes[i].adminState === "Decommission In Progress") {
@@ -249,6 +258,7 @@
             'lengthMenu': [ [25, 50, 100, -1], [25, 50, 100, "All"] ],
             'columns': [
               { 'orderDataType': 'ng-value', 'searchable': true },
+              { 'orderDataType': 'ng-value', 'searchable': true },
               { 'orderDataType': 'ng-value', 'type': 'numeric' },
               { 'orderDataType': 'ng-value', 'type': 'numeric' },
               { 'orderData': 3, 'type': 'numeric' },


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


[02/39] hadoop git commit: YARN-4887. Add allocation request ID to AM-RM protocol for identifying resource-requests explicitly. (Subru Krishnan via wangda)

Posted by ae...@apache.org.
YARN-4887. Add allocation request ID to AM-RM protocol for identifying resource-requests explicitly. (Subru Krishnan via wangda)


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

Branch: refs/heads/HDFS-1312
Commit: eec835ec17f77bf713d90dbc2f77dea45f3ee660
Parents: e3ba9ad
Author: Wangda Tan <wa...@apache.org>
Authored: Mon Jun 13 21:57:33 2016 -0700
Committer: Wangda Tan <wa...@apache.org>
Committed: Mon Jun 13 21:57:33 2016 -0700

----------------------------------------------------------------------
 .../hadoop/yarn/api/records/Container.java      | 45 +++++++++++++++++
 .../yarn/api/records/ResourceRequest.java       | 52 ++++++++++++++++++++
 .../src/main/proto/yarn_protos.proto            |  2 +
 .../api/records/impl/pb/ContainerPBImpl.java    | 14 ++++++
 .../records/impl/pb/ResourceRequestPBImpl.java  | 16 +++++-
 5 files changed, 128 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/eec835ec/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/Container.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/Container.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/Container.java
index 9a62935..707a71d 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/Container.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/Container.java
@@ -20,6 +20,7 @@ package org.apache.hadoop.yarn.api.records;
 
 import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.classification.InterfaceAudience.Public;
+import org.apache.hadoop.classification.InterfaceStability.Evolving;
 import org.apache.hadoop.classification.InterfaceStability.Stable;
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.yarn.api.ApplicationMasterProtocol;
@@ -189,4 +190,48 @@ public abstract class Container implements Comparable<Container> {
   @Private
   @Unstable
   public abstract void setExecutionType(ExecutionType executionType);
+
+  /**
+   * Get the optional <em>ID</em> corresponding to the original {@code
+   * ResourceRequest{@link #getAllocationRequestId()}}s which is satisfied by
+   * this allocated {@code Container}.
+   * <p>
+   * The scheduler may return multiple {@code AllocateResponse}s corresponding
+   * to the same ID as and when scheduler allocates {@code Container}s.
+   * <b>Applications</b> can continue to completely ignore the returned ID in
+   * the response and use the allocation for any of their outstanding requests.
+   * <p>
+   *
+   * @return the <em>ID</em> corresponding to the original  allocation request
+   * which is satisfied by this allocation.
+   */
+  @Public
+  @Evolving
+  public long getAllocationRequestId() {
+    throw new UnsupportedOperationException();
+  }
+
+  /**
+   * Set the optional <em>ID</em> corresponding to the original {@code
+   * ResourceRequest{@link #setAllocationRequestId(long)}
+   * etAllocationRequestId()}}s which is satisfied by this allocated {@code
+   * Container}.
+   * <p>
+   * The scheduler may return multiple {@code AllocateResponse}s corresponding
+   * to the same ID as and when scheduler allocates {@code Container}s.
+   * <b>Applications</b> can continue to completely ignore the returned ID in
+   * the response and use the allocation for any of their outstanding requests.
+   * If the ID is not set, scheduler will continue to work as previously and all
+   * allocated {@code Container}(s) will have the default ID, -1.
+   * <p>
+   *
+   * @param allocationRequestID the <em>ID</em> corresponding to the original
+   *                            allocation request which is satisfied by this
+   *                            allocation.
+   */
+  @Private
+  @Evolving
+  public void setAllocationRequestId(long allocationRequestID) {
+    throw new UnsupportedOperationException();
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/eec835ec/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ResourceRequest.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ResourceRequest.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ResourceRequest.java
index fbe7e58..50a7619 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ResourceRequest.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ResourceRequest.java
@@ -312,6 +312,58 @@ public abstract class ResourceRequest implements Comparable<ResourceRequest> {
   @Public
   @Evolving
   public abstract void setNodeLabelExpression(String nodelabelExpression);
+
+  /**
+   * Get the optional <em>ID</em> corresponding to this allocation request. This
+   * ID is an identifier for different {@code ResourceRequest}s from the <b>same
+   * application</b>. The allocated {@code Container}(s) received as part of the
+   * {@code AllocateResponse} response will have the ID corresponding to the
+   * original {@code ResourceRequest} for which the RM made the allocation.
+   * <p>
+   * The scheduler may return multiple {@code AllocateResponse}s corresponding
+   * to the same ID as and when scheduler allocates {@code Container}(s).
+   * <b>Applications</b> can continue to completely ignore the returned ID in
+   * the response and use the allocation for any of their outstanding requests.
+   * <p>
+   * If one wishes to replace an entire {@code ResourceRequest} corresponding to
+   * a specific ID, they can simply cancel the corresponding {@code
+   * ResourceRequest} and submit a new one afresh.
+   *
+   * @return the <em>ID</em> corresponding to this allocation request.
+   */
+  @Public
+  @Evolving
+  public long getAllocationRequestId() {
+    throw new UnsupportedOperationException();
+  }
+
+  /**
+   * Set the optional <em>ID</em> corresponding to this allocation request. This
+   * ID is an identifier for different {@code ResourceRequest}s from the <b>same
+   * application</b>. The allocated {@code Container}(s) received as part of the
+   * {@code AllocateResponse} response will have the ID corresponding to the
+   * original {@code ResourceRequest} for which the RM made the allocation.
+   * <p>
+   * The scheduler may return multiple {@code AllocateResponse}s corresponding
+   * to the same ID as and when scheduler allocates {@code Container}(s).
+   * <b>Applications</b> can continue to completely ignore the returned ID in
+   * the response and use the allocation for any of their outstanding requests.
+   * <p>
+   * If one wishes to replace an entire {@code ResourceRequest} corresponding to
+   * a specific ID, they can simply cancel the corresponding {@code
+   * ResourceRequest} and submit a new one afresh.
+   * <p>
+   * If the ID is not set, scheduler will continue to work as previously and all
+   * allocated {@code Container}(s) will have the default ID, -1.
+   *
+   * @param allocationRequestID the <em>ID</em> corresponding to this allocation
+   *                            request.
+   */
+  @Public
+  @Evolving
+  public void setAllocationRequestId(long allocationRequestID) {
+    throw new UnsupportedOperationException();
+  }
   
   @Override
   public int hashCode() {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/eec835ec/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto
index 0649f8e..c84f4e9 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto
@@ -93,6 +93,7 @@ message ContainerProto {
   optional PriorityProto priority = 5;
   optional hadoop.common.TokenProto container_token = 6;
   optional ExecutionTypeProto execution_type = 7 [default = GUARANTEED];
+  optional int64 allocation_request_id = 8 [default = -1];
 }
 
 message ContainerReportProto {
@@ -302,6 +303,7 @@ message ResourceRequestProto {
   optional bool relax_locality = 5 [default = true];
   optional string node_label_expression = 6;
   optional ExecutionTypeRequestProto execution_type_request = 7;
+  optional int64 allocation_request_id = 8 [default = -1];
 }
 
 message ExecutionTypeRequestProto {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/eec835ec/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ContainerPBImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ContainerPBImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ContainerPBImpl.java
index bd2d937..91b3e5f 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ContainerPBImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ContainerPBImpl.java
@@ -262,6 +262,18 @@ public class ContainerPBImpl extends Container {
     builder.setExecutionType(convertToProtoFormat(executionType));
   }
 
+  @Override
+  public long getAllocationRequestId() {
+    ContainerProtoOrBuilder p = viaProto ? proto : builder;
+    return (p.getAllocationRequestId());
+  }
+
+  @Override
+  public void setAllocationRequestId(long allocationRequestID) {
+    maybeInitBuilder();
+    builder.setAllocationRequestId(allocationRequestID);
+  }
+
   private ContainerIdPBImpl convertFromProtoFormat(ContainerIdProto p) {
     return new ContainerIdPBImpl(p);
   }
@@ -315,6 +327,8 @@ public class ContainerPBImpl extends Container {
     StringBuilder sb = new StringBuilder();
     sb.append("Container: [");
     sb.append("ContainerId: ").append(getId()).append(", ");
+    sb.append("AllocationRequestId: ").append(getAllocationRequestId())
+        .append(", ");
     sb.append("NodeId: ").append(getNodeId()).append(", ");
     sb.append("NodeHttpAddress: ").append(getNodeHttpAddress()).append(", ");
     sb.append("Resource: ").append(getResource()).append(", ");

http://git-wip-us.apache.org/repos/asf/hadoop/blob/eec835ec/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 b0c4b97..9890296 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
@@ -192,6 +192,18 @@ public class ResourceRequestPBImpl extends  ResourceRequest {
     builder.setRelaxLocality(relaxLocality);
   }
 
+  @Override
+  public long getAllocationRequestId() {
+    ResourceRequestProtoOrBuilder p = viaProto ? proto : builder;
+    return (p.getAllocationRequestId());
+  }
+
+  @Override
+  public void setAllocationRequestId(long allocationRequestID) {
+    maybeInitBuilder();
+    builder.setAllocationRequestId(allocationRequestID);
+  }
+
   private PriorityPBImpl convertFromProtoFormat(PriorityProto p) {
     return new PriorityPBImpl(p);
   }
@@ -210,7 +222,9 @@ public class ResourceRequestPBImpl extends  ResourceRequest {
   
   @Override
   public String toString() {
-    return "{Priority: " + getPriority() + ", Capability: " + getCapability()
+    return "{AllocationRequestId: " + getAllocationRequestId()
+        + ", Priority: " + getPriority()
+        + ", Capability: " + getCapability()
         + ", # Containers: " + getNumContainers()
         + ", Location: " + getResourceName()
         + ", Relax Locality: " + getRelaxLocality()


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


[04/39] hadoop git commit: HADOOP-13244. o.a.h.ipc.Server#Server should honor handlerCount when queueSizePerHandler is specified in consturctor. Contributed by Kai Sasaki.

Posted by ae...@apache.org.
HADOOP-13244. o.a.h.ipc.Server#Server should honor handlerCount when queueSizePerHandler is specified in consturctor. 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/20b13d10
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/20b13d10
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/20b13d10

Branch: refs/heads/HDFS-1312
Commit: 20b13d109a1f66d8911936d26bdde38d81a1c4c6
Parents: ee55b74
Author: Masatake Iwasaki <iw...@apache.org>
Authored: Tue Jun 14 19:20:46 2016 +0900
Committer: Masatake Iwasaki <iw...@apache.org>
Committed: Tue Jun 14 19:20:46 2016 +0900

----------------------------------------------------------------------
 .../src/main/java/org/apache/hadoop/ipc/Server.java              | 2 +-
 .../src/test/java/org/apache/hadoop/ipc/TestRPC.java             | 4 ++++
 2 files changed, 5 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/20b13d10/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Server.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Server.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Server.java
index 88c1f3c..caa534c 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Server.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Server.java
@@ -2553,7 +2553,7 @@ public abstract class Server {
     this.maxDataLength = conf.getInt(CommonConfigurationKeys.IPC_MAXIMUM_DATA_LENGTH,
         CommonConfigurationKeys.IPC_MAXIMUM_DATA_LENGTH_DEFAULT);
     if (queueSizePerHandler != -1) {
-      this.maxQueueSize = queueSizePerHandler;
+      this.maxQueueSize = handlerCount * queueSizePerHandler;
     } else {
       this.maxQueueSize = handlerCount * conf.getInt(
           CommonConfigurationKeys.IPC_SERVER_HANDLER_QUEUE_SIZE_KEY,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/20b13d10/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestRPC.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestRPC.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestRPC.java
index dbc9430..ff6b25e 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestRPC.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestRPC.java
@@ -348,6 +348,10 @@ public class TestRPC extends TestRpcBase {
 
     assertEquals(3, server.getNumReaders());
     assertEquals(200, server.getMaxQueueSize());
+
+    server = newServerBuilder(conf).setQueueSizePerHandler(10)
+        .setNumHandlers(2).setVerbose(false).build();
+    assertEquals(2 * 10, server.getMaxQueueSize());
   }
 
   @Test


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


[27/39] hadoop git commit: HADOOP-12975. Add jitter to CachingGetSpaceUsed's thread (Elliott Clark via Colin P. McCabe)

Posted by ae...@apache.org.
HADOOP-12975. Add jitter to CachingGetSpaceUsed's thread (Elliott Clark 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/bf780406
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/bf780406
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/bf780406

Branch: refs/heads/HDFS-1312
Commit: bf780406f2b30e627bdf36ac07973f6931f81106
Parents: b1674ca
Author: Colin Patrick Mccabe <cm...@cloudera.com>
Authored: Thu Jun 16 16:48:05 2016 -0700
Committer: Colin Patrick Mccabe <cm...@cloudera.com>
Committed: Thu Jun 16 16:48:05 2016 -0700

----------------------------------------------------------------------
 .../apache/hadoop/fs/CachingGetSpaceUsed.java   | 28 ++++++++++++++++----
 .../src/main/java/org/apache/hadoop/fs/DU.java  | 10 ++++---
 .../java/org/apache/hadoop/fs/GetSpaceUsed.java | 25 +++++++++++++++++
 .../apache/hadoop/fs/WindowsGetSpaceUsed.java   |  9 ++++---
 .../test/java/org/apache/hadoop/fs/TestDU.java  | 10 +++----
 5 files changed, 65 insertions(+), 17 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/bf780406/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CachingGetSpaceUsed.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CachingGetSpaceUsed.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CachingGetSpaceUsed.java
index 6ef75d2..505f76d 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CachingGetSpaceUsed.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CachingGetSpaceUsed.java
@@ -25,6 +25,7 @@ import org.slf4j.LoggerFactory;
 import java.io.Closeable;
 import java.io.File;
 import java.io.IOException;
+import java.util.concurrent.ThreadLocalRandom;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicLong;
 
@@ -43,6 +44,7 @@ public abstract class CachingGetSpaceUsed implements Closeable, GetSpaceUsed {
   protected final AtomicLong used = new AtomicLong();
   private final AtomicBoolean running = new AtomicBoolean(true);
   private final long refreshInterval;
+  private final long jitter;
   private final String dirPath;
   private Thread refreshUsed;
 
@@ -52,7 +54,10 @@ public abstract class CachingGetSpaceUsed implements Closeable, GetSpaceUsed {
    */
   public CachingGetSpaceUsed(CachingGetSpaceUsed.Builder builder)
       throws IOException {
-    this(builder.getPath(), builder.getInterval(), builder.getInitialUsed());
+    this(builder.getPath(),
+        builder.getInterval(),
+        builder.getJitter(),
+        builder.getInitialUsed());
   }
 
   /**
@@ -65,10 +70,12 @@ public abstract class CachingGetSpaceUsed implements Closeable, GetSpaceUsed {
    */
   CachingGetSpaceUsed(File path,
                       long interval,
+                      long jitter,
                       long initialUsed) throws IOException {
-    dirPath = path.getCanonicalPath();
-    refreshInterval = interval;
-    used.set(initialUsed);
+    this.dirPath = path.getCanonicalPath();
+    this.refreshInterval = interval;
+    this.jitter = jitter;
+    this.used.set(initialUsed);
   }
 
   void init() {
@@ -155,7 +162,18 @@ public abstract class CachingGetSpaceUsed implements Closeable, GetSpaceUsed {
     public void run() {
       while (spaceUsed.running()) {
         try {
-          Thread.sleep(spaceUsed.getRefreshInterval());
+          long refreshInterval = spaceUsed.refreshInterval;
+
+          if (spaceUsed.jitter > 0) {
+            long jitter = spaceUsed.jitter;
+            // add/subtract the jitter.
+            refreshInterval +=
+                ThreadLocalRandom.current()
+                                 .nextLong(-jitter, jitter);
+          }
+          // Make sure that after the jitter we didn't end up at 0.
+          refreshInterval = Math.max(refreshInterval, 1);
+          Thread.sleep(refreshInterval);
           // update the used variable
           spaceUsed.refresh();
         } catch (InterruptedException e) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bf780406/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/DU.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/DU.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/DU.java
index f700e4f..20e8202 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/DU.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/DU.java
@@ -34,12 +34,16 @@ public class DU extends CachingGetSpaceUsed {
   private DUShell duShell;
 
   @VisibleForTesting
-   public DU(File path, long interval, long initialUsed) throws IOException {
-    super(path, interval, initialUsed);
+  public DU(File path, long interval, long jitter, long initialUsed)
+      throws IOException {
+    super(path, interval, jitter, initialUsed);
   }
 
   public DU(CachingGetSpaceUsed.Builder builder) throws IOException {
-    this(builder.getPath(), builder.getInterval(), builder.getInitialUsed());
+    this(builder.getPath(),
+        builder.getInterval(),
+        builder.getJitter(),
+        builder.getInitialUsed());
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bf780406/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/GetSpaceUsed.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/GetSpaceUsed.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/GetSpaceUsed.java
index aebc3f7..4d1f9ef 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/GetSpaceUsed.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/GetSpaceUsed.java
@@ -26,8 +26,11 @@ import java.io.File;
 import java.io.IOException;
 import java.lang.reflect.Constructor;
 import java.lang.reflect.InvocationTargetException;
+import java.util.concurrent.TimeUnit;
 
 public interface GetSpaceUsed {
+
+
   long getUsed() throws IOException;
 
   /**
@@ -37,11 +40,15 @@ public interface GetSpaceUsed {
     static final Logger LOG = LoggerFactory.getLogger(Builder.class);
 
     static final String CLASSNAME_KEY = "fs.getspaceused.classname";
+    static final String JITTER_KEY = "fs.getspaceused.jitterMillis";
+    static final long DEFAULT_JITTER = TimeUnit.MINUTES.toMillis(1);
+
 
     private Configuration conf;
     private Class<? extends GetSpaceUsed> klass = null;
     private File path = null;
     private Long interval = null;
+    private Long jitter = null;
     private Long initialUsed = null;
 
     public Configuration getConf() {
@@ -111,6 +118,24 @@ public interface GetSpaceUsed {
       return this;
     }
 
+
+    public long getJitter() {
+      if (jitter == null) {
+        Configuration configuration = this.conf;
+
+        if (configuration == null) {
+          return DEFAULT_JITTER;
+        }
+        return configuration.getLong(JITTER_KEY, DEFAULT_JITTER);
+      }
+      return jitter;
+    }
+
+    public Builder setJitter(Long jit) {
+      this.jitter = jit;
+      return this;
+    }
+
     public GetSpaceUsed build() throws IOException {
       GetSpaceUsed getSpaceUsed = null;
       try {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bf780406/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/WindowsGetSpaceUsed.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/WindowsGetSpaceUsed.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/WindowsGetSpaceUsed.java
index 2796089..8e8c459 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/WindowsGetSpaceUsed.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/WindowsGetSpaceUsed.java
@@ -31,10 +31,11 @@ import java.io.IOException;
 @InterfaceStability.Evolving
 public class WindowsGetSpaceUsed extends CachingGetSpaceUsed {
 
-
-  public WindowsGetSpaceUsed(CachingGetSpaceUsed.Builder builder)
-      throws IOException {
-    super(builder.getPath(), builder.getInterval(), builder.getInitialUsed());
+  WindowsGetSpaceUsed(CachingGetSpaceUsed.Builder builder) throws IOException {
+    super(builder.getPath(),
+        builder.getInterval(),
+        builder.getJitter(),
+        builder.getInitialUsed());
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bf780406/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestDU.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestDU.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestDU.java
index 739263f..615d0b5 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestDU.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestDU.java
@@ -79,7 +79,7 @@ public class TestDU extends TestCase {
 
     Thread.sleep(5000); // let the metadata updater catch up
 
-    DU du = new DU(file, 10000, -1);
+    DU du = new DU(file, 10000, 0, -1);
     du.init();
     long duSize = du.getUsed();
     du.close();
@@ -89,7 +89,7 @@ public class TestDU extends TestCase {
         writtenSize <= (duSize + slack));
 
     //test with 0 interval, will not launch thread
-    du = new DU(file, 0, -1);
+    du = new DU(file, 0, 1, -1);
     du.init();
     duSize = du.getUsed();
     du.close();
@@ -99,7 +99,7 @@ public class TestDU extends TestCase {
         writtenSize <= (duSize + slack));
 
     //test without launching thread
-    du = new DU(file, 10000, -1);
+    du = new DU(file, 10000, 0, -1);
     du.init();
     duSize = du.getUsed();
 
@@ -112,7 +112,7 @@ public class TestDU extends TestCase {
     assertTrue(file.createNewFile());
     Configuration conf = new Configuration();
     conf.setLong(CommonConfigurationKeys.FS_DU_INTERVAL_KEY, 10000L);
-    DU du = new DU(file, 10000L, -1);
+    DU du = new DU(file, 10000L, 0, -1);
     du.incDfsUsed(-Long.MAX_VALUE);
     long duSize = du.getUsed();
     assertTrue(String.valueOf(duSize), duSize >= 0L);
@@ -121,7 +121,7 @@ public class TestDU extends TestCase {
   public void testDUSetInitialValue() throws IOException {
     File file = new File(DU_DIR, "dataX");
     createFile(file, 8192);
-    DU du = new DU(file, 3000, 1024);
+    DU du = new DU(file, 3000, 0, 1024);
     du.init();
     assertTrue("Initial usage setting not honored", du.getUsed() == 1024);
 


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


[31/39] hadoop git commit: HADOOP-12943. Add -w -r options in dfs -test command. Contributed by Weiwei Yang.

Posted by ae...@apache.org.
HADOOP-12943. Add -w -r options in dfs -test command. Contributed by Weiwei Yang.


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

Branch: refs/heads/HDFS-1312
Commit: 09e82acaf9a6d7663bc51bbca0cdeca4b582b535
Parents: 51d16e7
Author: Akira Ajisaka <aa...@apache.org>
Authored: Fri Jun 17 16:16:44 2016 +0900
Committer: Akira Ajisaka <aa...@apache.org>
Committed: Fri Jun 17 16:20:38 2016 +0900

----------------------------------------------------------------------
 .../java/org/apache/hadoop/fs/shell/Test.java   | 80 +++++++++++++-------
 .../src/site/markdown/FileSystemShell.md        |  3 +
 .../org/apache/hadoop/hdfs/TestDFSShell.java    | 71 ++++++++++++++++-
 3 files changed, 126 insertions(+), 28 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/09e82aca/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/Test.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/Test.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/Test.java
index 9984cf2..a2d2529 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/Test.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/Test.java
@@ -18,11 +18,14 @@
 
 package org.apache.hadoop.fs.shell;
 
+import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.util.LinkedList;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.fs.permission.FsAction;
+import org.apache.hadoop.security.AccessControlException;
 
 /**
  * Perform shell-like file tests 
@@ -38,18 +41,25 @@ class Test extends FsCommand {
   public static final String NAME = "test";
   public static final String USAGE = "-[defsz] <path>";
   public static final String DESCRIPTION =
-    "Answer various questions about <path>, with result via exit status.\n" +
-    "  -d  return 0 if <path> is a directory.\n" +
-    "  -e  return 0 if <path> exists.\n" +
-    "  -f  return 0 if <path> is a file.\n" +
-    "  -s  return 0 if file <path> is greater than zero bytes in size.\n" +
-    "  -z  return 0 if file <path> is zero bytes in size, else return 1.";
+      "Answer various questions about <path>, with result via exit status.\n"
+          + "  -d  return 0 if <path> is a directory.\n"
+          + "  -e  return 0 if <path> exists.\n"
+          + "  -f  return 0 if <path> is a file.\n"
+          + "  -s  return 0 if file <path> is greater "
+          + "        than zero bytes in size.\n"
+          + "  -w  return 0 if file <path> exists "
+          + "        and write permission is granted.\n"
+          + "  -r  return 0 if file <path> exists "
+          + "        and read permission is granted.\n"
+          + "  -z  return 0 if file <path> is "
+          + "        zero bytes in size, else return 1.";
 
   private char flag;
   
   @Override
   protected void processOptions(LinkedList<String> args) {
-    CommandFormat cf = new CommandFormat(1, 1, "e", "d", "f", "s", "z");
+    CommandFormat cf = new CommandFormat(1, 1,
+        "e", "d", "f", "s", "z", "w", "r");
     cf.parse(args);
     
     String[] opts = cf.getOpts().toArray(new String[0]);
@@ -68,29 +78,47 @@ class Test extends FsCommand {
   protected void processPath(PathData item) throws IOException {
     boolean test = false;
     switch (flag) {
-      case 'e':
-        test = true;
-        break;
-      case 'd':
-        test = item.stat.isDirectory();
-        break;
-      case 'f':
-        test = item.stat.isFile();
-        break;
-      case 's':
-        test = (item.stat.getLen() > 0);
-        break;
-      case 'z':
-        test = (item.stat.getLen() == 0);
-        break;
-      default:
-        break;
+    case 'e':
+      test = true;
+      break;
+    case 'd':
+      test = item.stat.isDirectory();
+      break;
+    case 'f':
+      test = item.stat.isFile();
+      break;
+    case 's':
+      test = (item.stat.getLen() > 0);
+      break;
+    case 'z':
+      test = (item.stat.getLen() == 0);
+      break;
+    case 'w':
+      test = testAccess(item, FsAction.WRITE);
+      break;
+    case 'r':
+      test = testAccess(item, FsAction.READ);
+      break;
+    default:
+      break;
+    }
+    if (!test) {
+      exitCode = 1;
+    }
+  }
+
+  private boolean testAccess(PathData item, FsAction action)
+      throws IOException {
+    try {
+      item.fs.access(item.path, action);
+      return true;
+    } catch (AccessControlException | FileNotFoundException e) {
+      return false;
     }
-    if (!test) exitCode = 1;
   }
 
   @Override
   protected void processNonexistentPath(PathData item) throws IOException {
     exitCode = 1;
   }
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/09e82aca/hadoop-common-project/hadoop-common/src/site/markdown/FileSystemShell.md
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/site/markdown/FileSystemShell.md b/hadoop-common-project/hadoop-common/src/site/markdown/FileSystemShell.md
index 1723426..066cfe3 100644
--- a/hadoop-common-project/hadoop-common/src/site/markdown/FileSystemShell.md
+++ b/hadoop-common-project/hadoop-common/src/site/markdown/FileSystemShell.md
@@ -669,8 +669,11 @@ Options:
 * -e: if the path exists, return 0.
 * -f: if the path is a file, return 0.
 * -s: if the path is not empty, return 0.
+* -r: if the path exists and read permission is granted, return 0.
+* -w: if the path exists and write permission is granted, return 0.
 * -z: if the file is zero length, return 0.
 
+
 Example:
 
 * `hadoop fs -test -e filename`

http://git-wip-us.apache.org/repos/asf/hadoop/blob/09e82aca/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 24d8b90..9cae762 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
@@ -1179,8 +1179,8 @@ public class TestDFSShell {
    * Tests various options of DFSShell.
    */
   @Test (timeout = 120000)
-  public void testDFSShell() throws IOException {
-    Configuration conf = new HdfsConfiguration();
+  public void testDFSShell() throws Exception {
+    final Configuration conf = new HdfsConfiguration();
     /* This tests some properties of ChecksumFileSystem as well.
      * Make sure that we create ChecksumDFS */
     MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(2).build();
@@ -1532,6 +1532,73 @@ public class TestDFSShell {
         assertEquals(0, val);
       }
 
+      // Verify -test -w/-r
+      {
+        Path permDir = new Path("/test/permDir");
+        Path permFile = new Path("/test/permDir/permFile");
+        mkdir(fs, permDir);
+        writeFile(fs, permFile);
+
+        // Verify -test -w positive case (dir exists and can write)
+        final String[] wargs = new String[3];
+        wargs[0] = "-test";
+        wargs[1] = "-w";
+        wargs[2] = permDir.toString();
+        int val = -1;
+        try {
+          val = shell.run(wargs);
+        } catch (Exception e) {
+          System.err.println("Exception raised from DFSShell.run " +
+              e.getLocalizedMessage());
+        }
+        assertEquals(0, val);
+
+        // Verify -test -r positive case (file exists and can read)
+        final String[] rargs = new String[3];
+        rargs[0] = "-test";
+        rargs[1] = "-r";
+        rargs[2] = permFile.toString();
+        try {
+          val = shell.run(rargs);
+        } catch (Exception e) {
+          System.err.println("Exception raised from DFSShell.run " +
+              e.getLocalizedMessage());
+        }
+        assertEquals(0, val);
+
+        // Verify -test -r negative case (file exists but cannot read)
+        runCmd(shell, "-chmod", "600", permFile.toString());
+
+        UserGroupInformation smokeUser =
+            UserGroupInformation.createUserForTesting("smokeUser",
+                new String[] {"hadoop"});
+        smokeUser.doAs(new PrivilegedExceptionAction<String>() {
+            @Override
+            public String run() throws Exception {
+              FsShell shell = new FsShell(conf);
+              int exitCode = shell.run(rargs);
+              assertEquals(1, exitCode);
+              return null;
+            }
+          });
+
+        // Verify -test -w negative case (dir exists but cannot write)
+        runCmd(shell, "-chown", "-R", "not_allowed", permDir.toString());
+        runCmd(shell, "-chmod", "-R", "700", permDir.toString());
+
+        smokeUser.doAs(new PrivilegedExceptionAction<String>() {
+          @Override
+          public String run() throws Exception {
+            FsShell shell = new FsShell(conf);
+            int exitCode = shell.run(wargs);
+            assertEquals(1, exitCode);
+            return null;
+          }
+        });
+
+        // cleanup
+        fs.delete(permDir, true);
+      }
     } finally {
       try {
         fileSys.close();


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