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/22 00:39:00 UTC

[01/17] hadoop git commit: YARN-4958. The file localization process should allow for wildcards to reduce the application footprint in the state store (Daniel Templeton via sjlee)

Repository: hadoop
Updated Branches:
  refs/heads/HDFS-1312 90a032653 -> 62e4dcd2d


YARN-4958. The file localization process should allow for wildcards to reduce the application footprint in the state store (Daniel Templeton via sjlee)


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

Branch: refs/heads/HDFS-1312
Commit: 5107a967fa2558deba11c33a326d4d2e5748f452
Parents: fc6b50c
Author: Sangjin Lee <sj...@twitter.com>
Authored: Mon Jun 20 09:56:53 2016 -0700
Committer: Sangjin Lee <sj...@twitter.com>
Committed: Mon Jun 20 09:56:53 2016 -0700

----------------------------------------------------------------------
 .../yarn/server/nodemanager/ContainerExecutor.java   | 15 +++++++++++++--
 1 file changed, 13 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/5107a967/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/ContainerExecutor.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/ContainerExecutor.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/ContainerExecutor.java
index 98171af..98d45f4 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/ContainerExecutor.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/ContainerExecutor.java
@@ -61,7 +61,7 @@ import org.apache.hadoop.util.Shell;
 import org.apache.hadoop.util.StringUtils;
 
 public abstract class ContainerExecutor implements Configurable {
-
+  private static final String WILDCARD = "*";
   private static final Log LOG = LogFactory.getLog(ContainerExecutor.class);
   final public static FsPermission TASK_LAUNCH_SCRIPT_PERMISSION =
     FsPermission.createImmutable((short) 0700);
@@ -281,7 +281,18 @@ public abstract class ContainerExecutor implements Configurable {
     if (resources != null) {
       for (Map.Entry<Path,List<String>> entry : resources.entrySet()) {
         for (String linkName : entry.getValue()) {
-          sb.symlink(entry.getKey(), new Path(linkName));
+          if (new Path(linkName).getName().equals(WILDCARD)) {
+            // If this is a wildcarded path, link to everything in the
+            // directory from the working directory
+            File directory = new File(entry.getKey().toString());
+
+            for (File wildLink : directory.listFiles()) {
+              sb.symlink(new Path(wildLink.toString()),
+                  new Path(wildLink.getName()));
+            }
+          } else {
+            sb.symlink(entry.getKey(), new Path(linkName));
+          }
         }
       }
     }


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


[12/17] hadoop git commit: HDFS-9530. ReservedSpace is not cleared for abandoned Blocks (Contributed by Brahma Reddy Battula)

Posted by ae...@apache.org.
HDFS-9530. ReservedSpace is not cleared for abandoned Blocks (Contributed by Brahma Reddy Battula)


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

Branch: refs/heads/HDFS-1312
Commit: f2ac132d6a21c215093b7f87acf2843ac8123716
Parents: 46f1602
Author: Brahma Reddy Battula <br...@apache.org>
Authored: Tue Jun 21 15:42:28 2016 +0530
Committer: Brahma Reddy Battula <br...@apache.org>
Committed: Tue Jun 21 15:42:28 2016 +0530

----------------------------------------------------------------------
 .../server/datanode/DataNodeFaultInjector.java  |  2 +
 .../hdfs/server/datanode/DataXceiver.java       |  3 ++
 .../datanode/fsdataset/impl/FsDatasetImpl.java  |  4 ++
 .../fsdataset/impl/TestSpaceReservation.java    | 48 ++++++++++++++++++++
 4 files changed, 57 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/f2ac132d/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNodeFaultInjector.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNodeFaultInjector.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNodeFaultInjector.java
index 7327420..4ecbdc0 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNodeFaultInjector.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNodeFaultInjector.java
@@ -53,4 +53,6 @@ public class DataNodeFaultInjector {
   public void stopSendingPacketDownstream() throws IOException {}
 
   public void noRegistration() throws IOException { }
+
+  public void failMirrorConnection() throws IOException { }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f2ac132d/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataXceiver.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataXceiver.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataXceiver.java
index d5dc328..829badd 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataXceiver.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataXceiver.java
@@ -739,6 +739,9 @@ class DataXceiver extends Receiver implements Runnable {
         mirrorTarget = NetUtils.createSocketAddr(mirrorNode);
         mirrorSock = datanode.newSocket();
         try {
+
+          DataNodeFaultInjector.get().failMirrorConnection();
+
           int timeoutValue = dnConf.socketTimeout +
               (HdfsConstants.READ_TIMEOUT_EXTENSION * targets.length);
           int writeTimeout = dnConf.socketWriteTimeout +

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f2ac132d/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java
index 243a0e2..b042297 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java
@@ -79,6 +79,7 @@ import org.apache.hadoop.hdfs.server.datanode.ReplicaAlreadyExistsException;
 import org.apache.hadoop.hdfs.server.datanode.ReplicaBeingWritten;
 import org.apache.hadoop.hdfs.server.datanode.ReplicaHandler;
 import org.apache.hadoop.hdfs.server.datanode.ReplicaInPipeline;
+import org.apache.hadoop.hdfs.server.datanode.ReplicaInPipelineInterface;
 import org.apache.hadoop.hdfs.server.datanode.ReplicaInfo;
 import org.apache.hadoop.hdfs.server.datanode.ReplicaNotFoundException;
 import org.apache.hadoop.hdfs.server.datanode.ReplicaUnderRecovery;
@@ -1955,6 +1956,9 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
           LOG.debug("Block file " + removing.getBlockFile().getName()
               + " is to be deleted");
         }
+        if (removing instanceof ReplicaInPipelineInterface) {
+          ((ReplicaInPipelineInterface) removing).releaseAllBytesReserved();
+        }
       }
 
       if (v.isTransientStorage()) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f2ac132d/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestSpaceReservation.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestSpaceReservation.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestSpaceReservation.java
index 6dbd299..fad5216 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestSpaceReservation.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestSpaceReservation.java
@@ -36,6 +36,7 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.*;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
+import org.apache.hadoop.hdfs.server.datanode.DataNodeFaultInjector;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsDatasetSpi;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeReference;
 import org.apache.hadoop.ipc.RemoteException;
@@ -77,6 +78,7 @@ public class TestSpaceReservation {
   private DFSClient client = null;
   FsVolumeReference singletonVolumeRef = null;
   FsVolumeImpl singletonVolume = null;
+  private DataNodeFaultInjector old = null;
 
   private static Random rand = new Random();
 
@@ -146,6 +148,9 @@ public class TestSpaceReservation {
       cluster.shutdown();
       cluster = null;
     }
+    if (old != null) {
+      DataNodeFaultInjector.set(old);
+    }
   }
 
   private void createFileAndTestSpaceReservation(
@@ -613,6 +618,49 @@ public class TestSpaceReservation {
     checkReservedSpace(expectedFile2Reserved);
   }
 
+  @Test(timeout = 30000)
+  public void testReservedSpaceForPipelineRecovery() throws Exception {
+    final short replication = 3;
+    startCluster(BLOCK_SIZE, replication, -1);
+
+    final String methodName = GenericTestUtils.getMethodName();
+    final Path file = new Path("/" + methodName + ".01.dat");
+
+    old = DataNodeFaultInjector.get();
+    // Fault injector to fail connection to mirror first time.
+    DataNodeFaultInjector.set(new DataNodeFaultInjector() {
+      private int tries = 0;
+
+      @Override
+      public void failMirrorConnection() throws IOException {
+        if (tries++ == 0) {
+          throw new IOException("Failing Mirror for space reservation");
+        }
+      }
+    });
+    // Write 1 byte to the file and kill the writer.
+    FSDataOutputStream os = fs.create(file, replication);
+    os.write(new byte[1]);
+    os.close();
+    // Ensure all space reserved for the replica was released on each
+    // DataNode.
+    cluster.triggerBlockReports();
+    for (final DataNode dn : cluster.getDataNodes()) {
+      try (FsDatasetSpi.FsVolumeReferences volumes =
+          dn.getFSDataset().getFsVolumeReferences()) {
+        final FsVolumeImpl volume = (FsVolumeImpl) volumes.get(0);
+        GenericTestUtils.waitFor(new Supplier<Boolean>() {
+          @Override
+          public Boolean get() {
+            LOG.info("dn " + dn.getDisplayName() + " space : "
+                + volume.getReservedForReplicas());
+            return (volume.getReservedForReplicas() == 0);
+          }
+        }, 100, Integer.MAX_VALUE); // Wait until the test times out.
+      }
+    }
+  }
+
   private void checkReservedSpace(final long expectedReserved) throws TimeoutException,
       InterruptedException, IOException {
     for (final DataNode dn : cluster.getDataNodes()) {


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


[14/17] hadoop git commit: MAPREDUCE-6719. The list of -libjars archives should be replaced with a wildcard in the distributed cache to reduce the application footprint in the state store (Daniel Templeton via sjlee)

Posted by ae...@apache.org.
MAPREDUCE-6719. The list of -libjars archives should be replaced with a wildcard in the distributed cache to reduce the application footprint in the state store (Daniel Templeton via sjlee)


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

Branch: refs/heads/HDFS-1312
Commit: 605b4b61364781fc99ed27035c793153a20d8f71
Parents: e15cd43
Author: Sangjin Lee <sj...@twitter.com>
Authored: Tue Jun 21 11:25:11 2016 -0700
Committer: Sangjin Lee <sj...@twitter.com>
Committed: Tue Jun 21 11:25:11 2016 -0700

----------------------------------------------------------------------
 .../apache/hadoop/mapreduce/v2/util/MRApps.java |  70 +++++++--
 .../java/org/apache/hadoop/mapreduce/Job.java   |   7 +-
 .../hadoop/mapreduce/JobResourceUploader.java   |  20 ++-
 .../hadoop/mapreduce/JobSubmissionFiles.java    |   4 +-
 .../apache/hadoop/mapreduce/JobSubmitter.java   |   6 +-
 .../ClientDistributedCacheManager.java          |  31 +++-
 .../mapreduce/filecache/DistributedCache.java   |  76 ++++++++--
 .../src/main/resources/mapred-default.xml       |  18 +++
 .../TestClientDistributedCacheManager.java      | 151 ++++++++++++++++---
 .../filecache/TestDistributedCache.java         | 132 ++++++++++++++++
 .../hadoop/mapred/TestLocalJobSubmission.java   |  34 +++--
 .../apache/hadoop/mapreduce/v2/TestMRJobs.java  |  48 ++++--
 12 files changed, 510 insertions(+), 87 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/605b4b61/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 31e4c0f..b800d31 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
@@ -300,12 +300,36 @@ public class MRApps extends Apps {
         for (URI u: withLinks) {
           Path p = new Path(u);
           FileSystem remoteFS = p.getFileSystem(conf);
+          String name = p.getName();
+          String wildcard = null;
+
+          // If the path is wildcarded, resolve its parent directory instead
+          if (name.equals(DistributedCache.WILDCARD)) {
+            wildcard = name;
+            p = p.getParent();
+          }
+
           p = remoteFS.resolvePath(p.makeQualified(remoteFS.getUri(),
               remoteFS.getWorkingDirectory()));
-          String name = (null == u.getFragment())
-              ? p.getName() : u.getFragment();
+
+          if ((wildcard != null) && (u.getFragment() != null)) {
+            throw new IOException("Invalid path URI: " + p + " - cannot "
+                + "contain both a URI fragment and a wildcard");
+          } else if (wildcard != null) {
+            name = p.getName() + Path.SEPARATOR + wildcard;
+          } else if (u.getFragment() != null) {
+            name = u.getFragment();
+          }
+
+          // If it's not a JAR, add it to the link lookup.
           if (!StringUtils.toLowerCase(name).endsWith(".jar")) {
-            linkLookup.put(p, name);
+            String old = linkLookup.put(p, name);
+
+            if ((old != null) && !name.equals(old)) {
+              LOG.warn("The same path is included more than once "
+                  + "with different links or wildcards: " + p + " [" +
+                  name + ", " + old + "]");
+            }
           }
         }
       }
@@ -559,16 +583,42 @@ public class MRApps extends Apps {
         URI u = uris[i];
         Path p = new Path(u);
         FileSystem remoteFS = p.getFileSystem(conf);
+        String linkName = null;
+
+        if (p.getName().equals(DistributedCache.WILDCARD)) {
+          p = p.getParent();
+          linkName = p.getName() + Path.SEPARATOR + DistributedCache.WILDCARD;
+        }
+
         p = remoteFS.resolvePath(p.makeQualified(remoteFS.getUri(),
             remoteFS.getWorkingDirectory()));
-        // Add URI fragment or just the filename
-        Path name = new Path((null == u.getFragment())
-          ? p.getName()
-          : u.getFragment());
-        if (name.isAbsolute()) {
-          throw new IllegalArgumentException("Resource name must be relative");
+
+        // If there's no wildcard, try using the fragment for the link
+        if (linkName == null) {
+          linkName = u.getFragment();
+
+          // Because we don't know what's in the fragment, we have to handle
+          // it with care.
+          if (linkName != null) {
+            Path linkPath = new Path(linkName);
+
+            if (linkPath.isAbsolute()) {
+              throw new IllegalArgumentException("Resource name must be "
+                  + "relative");
+            }
+
+            linkName = linkPath.toUri().getPath();
+          }
+        } else if (u.getFragment() != null) {
+          throw new IllegalArgumentException("Invalid path URI: " + p +
+              " - cannot contain both a URI fragment and a wildcard");
         }
-        String linkName = name.toUri().getPath();
+
+        // If there's no wildcard or fragment, just link to the file name
+        if (linkName == null) {
+          linkName = p.getName();
+        }
+
         LocalResource orig = localResources.get(linkName);
         if(orig != null && !orig.getResource().equals(URL.fromURI(p.toUri()))) {
           throw new InvalidJobConfException(

http://git-wip-us.apache.org/repos/asf/hadoop/blob/605b4b61/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/Job.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/Job.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/Job.java
index 481107f..33e820b 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/Job.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/Job.java
@@ -95,10 +95,13 @@ public class Job extends JobContextImpl implements JobContext {
   static final int DEFAULT_MONITOR_POLL_INTERVAL = 1000;
 
   public static final String USED_GENERIC_PARSER = 
-    "mapreduce.client.genericoptionsparser.used";
+      "mapreduce.client.genericoptionsparser.used";
   public static final String SUBMIT_REPLICATION = 
-    "mapreduce.client.submit.file.replication";
+      "mapreduce.client.submit.file.replication";
   public static final int DEFAULT_SUBMIT_REPLICATION = 10;
+  public static final String USE_WILDCARD_FOR_LIBJARS =
+      "mapreduce.client.libjars.wildcard";
+  public static final boolean DEFAULT_USE_WILDCARD_FOR_LIBJARS = true;
 
   @InterfaceStability.Evolving
   public static enum TaskStatusFilter { NONE, KILLED, FAILED, SUCCEEDED, ALL }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/605b4b61/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/JobResourceUploader.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/JobResourceUploader.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/JobResourceUploader.java
index f3e4d2f..90e1fba 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/JobResourceUploader.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/JobResourceUploader.java
@@ -19,10 +19,8 @@ package org.apache.hadoop.mapreduce;
 
 import java.io.FileNotFoundException;
 import java.io.IOException;
-import java.net.InetAddress;
 import java.net.URI;
 import java.net.URISyntaxException;
-import java.net.UnknownHostException;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -40,10 +38,12 @@ import org.apache.hadoop.mapreduce.filecache.DistributedCache;
 @InterfaceStability.Unstable
 class JobResourceUploader {
   protected static final Log LOG = LogFactory.getLog(JobResourceUploader.class);
-  private FileSystem jtFs;
+  private final boolean useWildcard;
+  private final FileSystem jtFs;
 
-  JobResourceUploader(FileSystem submitFs) {
+  JobResourceUploader(FileSystem submitFs, boolean useWildcard) {
     this.jtFs = submitFs;
+    this.useWildcard = useWildcard;
   }
 
   /**
@@ -126,8 +126,18 @@ class JobResourceUploader {
       for (String tmpjars : libjarsArr) {
         Path tmp = new Path(tmpjars);
         Path newPath = copyRemoteFiles(libjarsDir, tmp, conf, replication);
+
+        // Add each file to the classpath
         DistributedCache.addFileToClassPath(
-            new Path(newPath.toUri().getPath()), conf, jtFs);
+            new Path(newPath.toUri().getPath()), conf, jtFs, !useWildcard);
+      }
+
+      if (useWildcard) {
+        // Add the whole directory to the cache
+        Path libJarsDirWildcard =
+            jtFs.makeQualified(new Path(libjarsDir, DistributedCache.WILDCARD));
+
+        DistributedCache.addCacheFile(libJarsDirWildcard.toUri(), conf);
       }
     }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/605b4b61/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/JobSubmissionFiles.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/JobSubmissionFiles.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/JobSubmissionFiles.java
index 7125077..c4adadf 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/JobSubmissionFiles.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/JobSubmissionFiles.java
@@ -41,10 +41,10 @@ public class JobSubmissionFiles {
 
   // job submission directory is private!
   final public static FsPermission JOB_DIR_PERMISSION =
-    FsPermission.createImmutable((short) 0700); // rwx--------
+      FsPermission.createImmutable((short) 0700); // rwx------
   //job files are world-wide readable and owner writable
   final public static FsPermission JOB_FILE_PERMISSION = 
-    FsPermission.createImmutable((short) 0644); // rw-r--r--
+      FsPermission.createImmutable((short) 0644); // rw-r--r--
   
   public static Path getJobSplitFile(Path jobSubmissionDir) {
     return new Path(jobSubmissionDir, "job.split");

http://git-wip-us.apache.org/repos/asf/hadoop/blob/605b4b61/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/JobSubmitter.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/JobSubmitter.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/JobSubmitter.java
index 497b0ed..22874e1 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/JobSubmitter.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/JobSubmitter.java
@@ -94,7 +94,11 @@ class JobSubmitter {
    */
   private void copyAndConfigureFiles(Job job, Path jobSubmitDir) 
   throws IOException {
-    JobResourceUploader rUploader = new JobResourceUploader(jtFs);
+    Configuration conf = job.getConfiguration();
+    boolean useWildcards = conf.getBoolean(Job.USE_WILDCARD_FOR_LIBJARS,
+        Job.DEFAULT_USE_WILDCARD_FOR_LIBJARS);
+    JobResourceUploader rUploader = new JobResourceUploader(jtFs, useWildcards);
+
     rUploader.uploadFiles(job, jobSubmitDir);
 
     // Get the working directory. If not set, sets it to filesystem working dir

http://git-wip-us.apache.org/repos/asf/hadoop/blob/605b4b61/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/filecache/ClientDistributedCacheManager.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/filecache/ClientDistributedCacheManager.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/filecache/ClientDistributedCacheManager.java
index c15e647..19470e8 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/filecache/ClientDistributedCacheManager.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/filecache/ClientDistributedCacheManager.java
@@ -227,21 +227,27 @@ public class ClientDistributedCacheManager {
   /**
    * Returns a boolean to denote whether a cache file is visible to all(public)
    * or not
-   * @param conf
-   * @param uri
+   * @param conf the configuration
+   * @param uri the URI to test
    * @return true if the path in the uri is visible to all, false otherwise
-   * @throws IOException
+   * @throws IOException thrown if a file system operation fails
    */
   static boolean isPublic(Configuration conf, URI uri,
       Map<URI, FileStatus> statCache) throws IOException {
+    boolean isPublic = true;
     FileSystem fs = FileSystem.get(uri, conf);
     Path current = new Path(uri.getPath());
     current = fs.makeQualified(current);
-    //the leaf level file should be readable by others
-    if (!checkPermissionOfOther(fs, current, FsAction.READ, statCache)) {
-      return false;
+
+    // If we're looking at a wildcarded path, we only need to check that the
+    // ancestors allow execution.  Otherwise, look for read permissions in
+    // addition to the ancestors' permissions.
+    if (!current.getName().equals(DistributedCache.WILDCARD)) {
+      isPublic = checkPermissionOfOther(fs, current, FsAction.READ, statCache);
     }
-    return ancestorsHaveExecutePermissions(fs, current.getParent(), statCache);
+
+    return isPublic &&
+        ancestorsHaveExecutePermissions(fs, current.getParent(), statCache);
   }
 
   /**
@@ -284,11 +290,20 @@ public class ClientDistributedCacheManager {
 
   private static FileStatus getFileStatus(FileSystem fs, URI uri,
       Map<URI, FileStatus> statCache) throws IOException {
+    Path path = new Path(uri);
+
+    if (path.getName().equals(DistributedCache.WILDCARD)) {
+      path = path.getParent();
+      uri = path.toUri();
+    }
+
     FileStatus stat = statCache.get(uri);
+
     if (stat == null) {
-      stat = fs.getFileStatus(new Path(uri));
+      stat = fs.getFileStatus(path);
       statCache.put(uri, stat);
     }
+
     return stat;
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/605b4b61/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/filecache/DistributedCache.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/filecache/DistributedCache.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/filecache/DistributedCache.java
index 51fe69a..d4d6c6e 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/filecache/DistributedCache.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/filecache/DistributedCache.java
@@ -126,12 +126,14 @@ import java.net.URI;
  * as well as methods intended for use by the MapReduce framework
  * (e.g., {@link org.apache.hadoop.mapred.JobClient}).
  *
+ * @see org.apache.hadoop.mapreduce.Job
  * @see org.apache.hadoop.mapred.JobConf
  * @see org.apache.hadoop.mapred.JobClient
  */
 @Deprecated
 @InterfaceAudience.Private
 public class DistributedCache {
+  public static final String WILDCARD = "*";
   
   /**
    * Set the configuration with the given set of archives.  Intended
@@ -139,6 +141,7 @@ public class DistributedCache {
    * @param archives The list of archives that need to be localized
    * @param conf Configuration which will be changed
    * @deprecated Use {@link Job#setCacheArchives(URI[])} instead
+   * @see Job#setCacheArchives(URI[])
    */
   @Deprecated
   public static void setCacheArchives(URI[] archives, Configuration conf) {
@@ -152,6 +155,7 @@ public class DistributedCache {
    * @param files The list of files that need to be localized
    * @param conf Configuration which will be changed
    * @deprecated Use {@link Job#setCacheFiles(URI[])} instead
+   * @see Job#setCacheFiles(URI[])
    */
   @Deprecated
   public static void setCacheFiles(URI[] files, Configuration conf) {
@@ -166,6 +170,7 @@ public class DistributedCache {
    * @return A URI array of the caches set in the Configuration
    * @throws IOException
    * @deprecated Use {@link JobContext#getCacheArchives()} instead
+   * @see JobContext#getCacheArchives()
    */
   @Deprecated
   public static URI[] getCacheArchives(Configuration conf) throws IOException {
@@ -179,6 +184,7 @@ public class DistributedCache {
    * @return A URI array of the files set in the Configuration
    * @throws IOException
    * @deprecated Use {@link JobContext#getCacheFiles()} instead
+   * @see JobContext#getCacheFiles()
    */
   @Deprecated
   public static URI[] getCacheFiles(Configuration conf) throws IOException {
@@ -192,6 +198,7 @@ public class DistributedCache {
    * @return A path array of localized caches
    * @throws IOException
    * @deprecated Use {@link JobContext#getLocalCacheArchives()} instead
+   * @see JobContext#getLocalCacheArchives()
    */
   @Deprecated
   public static Path[] getLocalCacheArchives(Configuration conf)
@@ -207,6 +214,7 @@ public class DistributedCache {
    * @return A path array of localized files
    * @throws IOException
    * @deprecated Use {@link JobContext#getLocalCacheFiles()} instead
+   * @see JobContext#getLocalCacheFiles()
    */
   @Deprecated
   public static Path[] getLocalCacheFiles(Configuration conf)
@@ -236,6 +244,7 @@ public class DistributedCache {
    * @param conf The configuration which stored the timestamps
    * @return a long array of timestamps
    * @deprecated Use {@link JobContext#getArchiveTimestamps()} instead
+   * @see JobContext#getArchiveTimestamps()
    */
   @Deprecated
   public static long[] getArchiveTimestamps(Configuration conf) {
@@ -250,6 +259,7 @@ public class DistributedCache {
    * @param conf The configuration which stored the timestamps
    * @return a long array of timestamps
    * @deprecated Use {@link JobContext#getFileTimestamps()} instead
+   * @see JobContext#getFileTimestamps()
    */
   @Deprecated
   public static long[] getFileTimestamps(Configuration conf) {
@@ -263,6 +273,7 @@ public class DistributedCache {
    * @param uri The uri of the cache to be localized
    * @param conf Configuration to add the cache to
    * @deprecated Use {@link Job#addCacheArchive(URI)} instead
+   * @see Job#addCacheArchive(URI)
    */
   @Deprecated
   public static void addCacheArchive(URI uri, Configuration conf) {
@@ -272,11 +283,27 @@ public class DistributedCache {
   }
   
   /**
-   * Add a file to be localized to the conf.  Intended
-   * to be used by user code.
+   * Add a file to be localized to the conf.  The localized file will be
+   * downloaded to the execution node(s), and a link will created to the
+   * file from the job's working directory. If the last part of URI's path name
+   * is "*", then the entire parent directory will be localized and links
+   * will be created from the job's working directory to each file in the
+   * parent directory.
+   *
+   * The access permissions of the file will determine whether the localized
+   * file will be shared across jobs.  If the file is not readable by other or
+   * if any of its parent directories is not executable by other, then the
+   * file will not be shared.  In the case of a path that ends in "/*",
+   * sharing of the localized files will be determined solely from the
+   * access permissions of the parent directories.  The access permissions of
+   * the individual files will be ignored.
+   *
+   * Intended to be used by user code.
+   *
    * @param uri The uri of the cache to be localized
    * @param conf Configuration to add the cache to
    * @deprecated Use {@link Job#addCacheFile(URI)} instead
+   * @see Job#addCacheFile(URI)
    */
   @Deprecated
   public static void addCacheFile(URI uri, Configuration conf) {
@@ -286,12 +313,14 @@ public class DistributedCache {
   }
 
   /**
-   * Add an file path to the current set of classpath entries It adds the file
-   * to cache as well.  Intended to be used by user code.
+   * Add a file path to the current set of classpath entries.  The file will
+   * also be added to the cache.  Intended to be used by user code.
    *
    * @param file Path of the file to be added
    * @param conf Configuration that contains the classpath setting
    * @deprecated Use {@link Job#addFileToClassPath(Path)} instead
+   * @see #addCacheFile(URI, Configuration)
+   * @see Job#addFileToClassPath(Path)
    */
   @Deprecated
   public static void addFileToClassPath(Path file, Configuration conf)
@@ -300,22 +329,42 @@ public class DistributedCache {
   }
 
   /**
-   * Add a file path to the current set of classpath entries. It adds the file
-   * to cache as well.  Intended to be used by user code.
+   * Add a file path to the current set of classpath entries. The file will
+   * also be added to the cache.  Intended to be used by user code.
    *
    * @param file Path of the file to be added
    * @param conf Configuration that contains the classpath setting
    * @param fs FileSystem with respect to which {@code archivefile} should
    *              be interpreted.
+   * @see #addCacheFile(URI, Configuration)
    */
-  public static void addFileToClassPath
-           (Path file, Configuration conf, FileSystem fs)
-        throws IOException {
+  public static void addFileToClassPath(Path file, Configuration conf,
+      FileSystem fs) {
+    addFileToClassPath(file, conf, fs, true);
+  }
+
+  /**
+   * Add a file path to the current set of classpath entries. The file will
+   * also be added to the cache if {@code addToCache} is true.  Used by
+   * internal DistributedCache code.
+   *
+   * @param file Path of the file to be added
+   * @param conf Configuration that contains the classpath setting
+   * @param fs FileSystem with respect to which {@code archivefile} should
+   *              be interpreted.
+   * @param addToCache whether the file should also be added to the cache list
+   * @see #addCacheFile(URI, Configuration)
+   */
+  public static void addFileToClassPath(Path file, Configuration conf,
+      FileSystem fs, boolean addToCache) {
     String classpath = conf.get(MRJobConfig.CLASSPATH_FILES);
     conf.set(MRJobConfig.CLASSPATH_FILES, classpath == null ? file.toString()
              : classpath + "," + file.toString());
-    URI uri = fs.makeQualified(file).toUri();
-    addCacheFile(uri, conf);
+
+    if (addToCache) {
+      URI uri = fs.makeQualified(file).toUri();
+      addCacheFile(uri, conf);
+    }
   }
 
   /**
@@ -323,7 +372,8 @@ public class DistributedCache {
    * Used by internal DistributedCache code.
    * 
    * @param conf Configuration that contains the classpath setting
-   * @deprecated Use {@link JobContext#getFileClassPaths()} instead 
+   * @deprecated Use {@link JobContext#getFileClassPaths()} instead
+   * @see JobContext#getFileClassPaths()
    */
   @Deprecated
   public static Path[] getFileClassPaths(Configuration conf) {
@@ -346,6 +396,7 @@ public class DistributedCache {
    * @param archive Path of the archive to be added
    * @param conf Configuration that contains the classpath setting
    * @deprecated Use {@link Job#addArchiveToClassPath(Path)} instead
+   * @see Job#addArchiveToClassPath(Path)
    */
   @Deprecated
   public static void addArchiveToClassPath(Path archive, Configuration conf)
@@ -378,6 +429,7 @@ public class DistributedCache {
    * 
    * @param conf Configuration that contains the classpath setting
    * @deprecated Use {@link JobContext#getArchiveClassPaths()} instead 
+   * @see JobContext#getArchiveClassPaths()
    */
   @Deprecated
   public static Path[] getArchiveClassPaths(Configuration conf) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/605b4b61/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/resources/mapred-default.xml
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/resources/mapred-default.xml b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/resources/mapred-default.xml
index 7634331..d973bd4 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/resources/mapred-default.xml
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/resources/mapred-default.xml
@@ -842,6 +842,24 @@
     </description>
   </property>
 
+  <property>
+    <name>mapreduce.client.libjars.wildcard</name>
+    <value>true</value>
+    <description>
+        Whether the libjars cache files should be localized using
+        a wildcarded directory instead of naming each archive independently.
+        Using wildcards reduces the space needed for storing the job
+        information in the case of a highly available resource manager
+        configuration.
+        This propery should only be set to false for specific
+        jobs which are highly sensitive to the details of the archive
+        localization.  Having this property set to true will cause the archives
+        to all be localized to the same local cache location.  If false, each
+        archive will be localized to its own local cache location.  In both
+        cases a symbolic link will be created to every archive from the job's
+        working directory.
+    </description>
+  </property>
 
   <property>
     <name>mapreduce.task.profile</name>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/605b4b61/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/filecache/TestClientDistributedCacheManager.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/filecache/TestClientDistributedCacheManager.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/filecache/TestClientDistributedCacheManager.java
index b5f45e6..5212b9f 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/filecache/TestClientDistributedCacheManager.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/filecache/TestClientDistributedCacheManager.java
@@ -17,11 +17,12 @@
  */
 package org.apache.hadoop.mapreduce.filecache;
 
-import java.io.File;
 import java.io.IOException;
 import java.net.URI;
 import java.util.HashMap;
+import java.util.HashSet;
 import java.util.Map;
+import java.util.Set;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -37,6 +38,8 @@ import org.apache.hadoop.mapreduce.Job;
 import org.apache.hadoop.mapreduce.MRJobConfig;
 import org.junit.After;
 import org.junit.Assert;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
 import org.junit.Before;
 import org.junit.Test;
 
@@ -55,22 +58,22 @@ public class TestClientDistributedCacheManager {
   private static final Path TEST_VISIBILITY_CHILD_DIR =
       new Path(TEST_VISIBILITY_PARENT_DIR, "TestCacheVisibility_Child");
 
+  private static final String FIRST_CACHE_FILE = "firstcachefile";
+  private static final String SECOND_CACHE_FILE = "secondcachefile";
+
   private FileSystem fs;
   private Path firstCacheFile;
   private Path secondCacheFile;
-  private Path thirdCacheFile;
   private Configuration conf;
   
   @Before
   public void setup() throws IOException {
     conf = new Configuration();
     fs = FileSystem.get(conf);
-    firstCacheFile = new Path(TEST_ROOT_DIR, "firstcachefile");
-    secondCacheFile = new Path(TEST_ROOT_DIR, "secondcachefile");
-    thirdCacheFile = new Path(TEST_VISIBILITY_CHILD_DIR,"thirdCachefile");
+    firstCacheFile = new Path(TEST_VISIBILITY_PARENT_DIR, FIRST_CACHE_FILE);
+    secondCacheFile = new Path(TEST_VISIBILITY_CHILD_DIR, SECOND_CACHE_FILE);
     createTempFile(firstCacheFile, conf);
     createTempFile(secondCacheFile, conf);
-    createTempFile(thirdCacheFile, conf);
   }
   
   @After
@@ -88,37 +91,147 @@ public class TestClientDistributedCacheManager {
     job.addCacheFile(secondCacheFile.toUri());
     Configuration jobConf = job.getConfiguration();
     
-    Map<URI, FileStatus> statCache = new HashMap<URI, FileStatus>();
+    Map<URI, FileStatus> statCache = new HashMap<>();
     ClientDistributedCacheManager.determineTimestamps(jobConf, statCache);
     
     FileStatus firstStatus = statCache.get(firstCacheFile.toUri());
     FileStatus secondStatus = statCache.get(secondCacheFile.toUri());
     
-    Assert.assertNotNull(firstStatus);
-    Assert.assertNotNull(secondStatus);
-    Assert.assertEquals(2, statCache.size());
+    Assert.assertNotNull(firstCacheFile + " was not found in the stats cache",
+        firstStatus);
+    Assert.assertNotNull(secondCacheFile + " was not found in the stats cache",
+        secondStatus);
+    Assert.assertEquals("Missing/extra entries found in the stas cache",
+        2, statCache.size());
     String expected = firstStatus.getModificationTime() + ","
         + secondStatus.getModificationTime();
     Assert.assertEquals(expected, jobConf.get(MRJobConfig.CACHE_FILE_TIMESTAMPS));
+
+    job = Job.getInstance(conf);
+    job.addCacheFile(new Path(TEST_VISIBILITY_CHILD_DIR, "*").toUri());
+    jobConf = job.getConfiguration();
+    statCache.clear();
+    ClientDistributedCacheManager.determineTimestamps(jobConf, statCache);
+
+    FileStatus thirdStatus = statCache.get(TEST_VISIBILITY_CHILD_DIR.toUri());
+
+    Assert.assertEquals("Missing/extra entries found in the stas cache",
+        1, statCache.size());
+    Assert.assertNotNull(TEST_VISIBILITY_CHILD_DIR
+        + " was not found in the stats cache", thirdStatus);
+    expected = Long.toString(thirdStatus.getModificationTime());
+    Assert.assertEquals("Incorrect timestamp for " + TEST_VISIBILITY_CHILD_DIR,
+        expected, jobConf.get(MRJobConfig.CACHE_FILE_TIMESTAMPS));
   }
   
   @Test
   public void testDetermineCacheVisibilities() throws IOException {
-    fs.setWorkingDirectory(TEST_VISIBILITY_CHILD_DIR);
+    fs.setPermission(TEST_VISIBILITY_PARENT_DIR,
+        new FsPermission((short)00777));
     fs.setPermission(TEST_VISIBILITY_CHILD_DIR,
         new FsPermission((short)00777));
+    fs.setWorkingDirectory(TEST_VISIBILITY_CHILD_DIR);
+    Job job = Job.getInstance(conf);
+    Path relativePath = new Path(SECOND_CACHE_FILE);
+    Path wildcardPath = new Path("*");
+    Map<URI, FileStatus> statCache = new HashMap<>();
+    Configuration jobConf;
+
+    job.addCacheFile(firstCacheFile.toUri());
+    job.addCacheFile(relativePath.toUri());
+    jobConf = job.getConfiguration();
+
+    ClientDistributedCacheManager.determineCacheVisibilities(jobConf,
+        statCache);
+    // We use get() instead of getBoolean() so we can tell the difference
+    // between wrong and missing
+    assertEquals("The file paths were not found to be publicly visible "
+        + "even though the full path is publicly accessible",
+        "true,true", jobConf.get(MRJobConfig.CACHE_FILE_VISIBILITIES));
+    checkCacheEntries(statCache, null, firstCacheFile, relativePath);
+
+    job = Job.getInstance(conf);
+    job.addCacheFile(wildcardPath.toUri());
+    jobConf = job.getConfiguration();
+    statCache.clear();
+
+    ClientDistributedCacheManager.determineCacheVisibilities(jobConf,
+        statCache);
+    // We use get() instead of getBoolean() so we can tell the difference
+    // between wrong and missing
+    assertEquals("The file path was not found to be publicly visible "
+        + "even though the full path is publicly accessible",
+        "true", jobConf.get(MRJobConfig.CACHE_FILE_VISIBILITIES));
+    checkCacheEntries(statCache, null, wildcardPath.getParent());
+
+    Path qualifiedParent = fs.makeQualified(TEST_VISIBILITY_PARENT_DIR);
     fs.setPermission(TEST_VISIBILITY_PARENT_DIR,
         new FsPermission((short)00700));
-    Job job = Job.getInstance(conf);
-    Path relativePath = new Path("thirdCachefile");
+    job = Job.getInstance(conf);
+    job.addCacheFile(firstCacheFile.toUri());
     job.addCacheFile(relativePath.toUri());
-    Configuration jobConf = job.getConfiguration();
+    jobConf = job.getConfiguration();
+    statCache.clear();
+
+    ClientDistributedCacheManager.determineCacheVisibilities(jobConf,
+        statCache);
+    // We use get() instead of getBoolean() so we can tell the difference
+    // between wrong and missing
+    assertEquals("The file paths were found to be publicly visible "
+        + "even though the parent directory is not publicly accessible",
+        "false,false", jobConf.get(MRJobConfig.CACHE_FILE_VISIBILITIES));
+    checkCacheEntries(statCache, qualifiedParent,
+        firstCacheFile, relativePath);
+
+    job = Job.getInstance(conf);
+    job.addCacheFile(wildcardPath.toUri());
+    jobConf = job.getConfiguration();
+    statCache.clear();
+
+    ClientDistributedCacheManager.determineCacheVisibilities(jobConf,
+        statCache);
+    // We use get() instead of getBoolean() so we can tell the difference
+    // between wrong and missing
+    assertEquals("The file path was found to be publicly visible "
+        + "even though the parent directory is not publicly accessible",
+        "false", jobConf.get(MRJobConfig.CACHE_FILE_VISIBILITIES));
+    checkCacheEntries(statCache, qualifiedParent, wildcardPath.getParent());
+  }
+
+  /**
+   * Validate that the file status cache contains all and only entries for a
+   * given set of paths up to a common parent.
+   *
+   * @param statCache the cache
+   * @param top the common parent at which to stop digging
+   * @param paths the paths to compare against the cache
+   */
+  private void checkCacheEntries(Map<URI, FileStatus> statCache, Path top,
+      Path... paths) {
+    Set<URI> expected = new HashSet<>();
+
+    for (Path path : paths) {
+      Path p = fs.makeQualified(path);
+
+      while (!p.isRoot() && !p.equals(top)) {
+        expected.add(p.toUri());
+        p = p.getParent();
+      }
+
+      expected.add(p.toUri());
+    }
+
+    Set<URI> uris = statCache.keySet();
+    Set<URI> missing = new HashSet<>(uris);
+    Set<URI> extra = new HashSet<>(expected);
+
+    missing.removeAll(expected);
+    extra.removeAll(uris);
 
-    Map<URI, FileStatus> statCache = new HashMap<URI, FileStatus>();
-    ClientDistributedCacheManager.
-        determineCacheVisibilities(jobConf, statCache);
-    Assert.assertFalse(jobConf.
-               getBoolean(MRJobConfig.CACHE_FILE_VISIBILITIES,true));
+    assertTrue("File status cache does not contain an entries for " + missing,
+        missing.isEmpty());
+    assertTrue("File status cache contains extra extries: " + extra,
+        extra.isEmpty());
   }
 
   @SuppressWarnings("deprecation")

http://git-wip-us.apache.org/repos/asf/hadoop/blob/605b4b61/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/filecache/TestDistributedCache.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/filecache/TestDistributedCache.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/filecache/TestDistributedCache.java
new file mode 100644
index 0000000..14f4020
--- /dev/null
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/filecache/TestDistributedCache.java
@@ -0,0 +1,132 @@
+/*
+ * 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.mapreduce.filecache;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.mapreduce.MRJobConfig;
+import org.junit.Test;
+import static org.junit.Assert.*;
+
+/**
+ * Test the {@link DistributedCache} class.
+ */
+public class TestDistributedCache {
+  /**
+   * Test of addFileOnlyToClassPath method, of class DistributedCache.
+   */
+  @Test
+  public void testAddFileToClassPath() throws Exception {
+    Configuration conf = new Configuration(false);
+
+    // Test first with 2 args
+    try {
+      DistributedCache.addFileToClassPath(null, conf);
+      fail("Accepted null archives argument");
+    } catch (NullPointerException ex) {
+      // Expected
+    }
+
+    DistributedCache.addFileToClassPath(new Path("file:///a"), conf);
+    assertEquals("The mapreduce.job.classpath.files property was not "
+        + "set correctly", "file:/a", conf.get(MRJobConfig.CLASSPATH_FILES));
+    assertEquals("The mapreduce.job.cache.files property was not set "
+        + "correctly", "file:///a", conf.get(MRJobConfig.CACHE_FILES));
+
+    DistributedCache.addFileToClassPath(new Path("file:///b"), conf);
+    assertEquals("The mapreduce.job.classpath.files property was not "
+        + "set correctly", "file:/a,file:/b",
+        conf.get(MRJobConfig.CLASSPATH_FILES));
+    assertEquals("The mapreduce.job.cache.files property was not set "
+        + "correctly", "file:///a,file:///b",
+        conf.get(MRJobConfig.CACHE_FILES));
+
+    // Now test with 3 args
+    FileSystem fs = FileSystem.newInstance(conf);
+    conf.clear();
+
+    try {
+      DistributedCache.addFileToClassPath(null, conf, fs);
+      fail("Accepted null archives argument");
+    } catch (NullPointerException ex) {
+      // Expected
+    }
+
+    DistributedCache.addFileToClassPath(new Path("file:///a"), conf, fs);
+    assertEquals("The mapreduce.job.classpath.files property was not "
+        + "set correctly", "file:/a", conf.get(MRJobConfig.CLASSPATH_FILES));
+    assertEquals("The mapreduce.job.cache.files property was not set "
+        + "correctly", "file:///a", conf.get(MRJobConfig.CACHE_FILES));
+
+    DistributedCache.addFileToClassPath(new Path("file:///b"), conf, fs);
+    assertEquals("The mapreduce.job.classpath.files property was not "
+        + "set correctly", "file:/a,file:/b",
+        conf.get(MRJobConfig.CLASSPATH_FILES));
+    assertEquals("The mapreduce.job.cache.files property was not set "
+        + "correctly", "file:///a,file:///b",
+        conf.get(MRJobConfig.CACHE_FILES));
+
+    // Now test with 4th arg true
+    conf.clear();
+
+    try {
+      DistributedCache.addFileToClassPath(null, conf, fs, true);
+      fail("Accepted null archives argument");
+    } catch (NullPointerException ex) {
+      // Expected
+    }
+
+    DistributedCache.addFileToClassPath(new Path("file:///a"), conf, fs, true);
+    assertEquals("The mapreduce.job.classpath.files property was not "
+        + "set correctly", "file:/a", conf.get(MRJobConfig.CLASSPATH_FILES));
+    assertEquals("The mapreduce.job.cache.files property was not set "
+        + "correctly", "file:///a", conf.get(MRJobConfig.CACHE_FILES));
+
+    DistributedCache.addFileToClassPath(new Path("file:///b"), conf, fs, true);
+    assertEquals("The mapreduce.job.classpath.files property was not "
+        + "set correctly", "file:/a,file:/b",
+        conf.get(MRJobConfig.CLASSPATH_FILES));
+    assertEquals("The mapreduce.job.cache.files property was not set "
+        + "correctly", "file:///a,file:///b",
+        conf.get(MRJobConfig.CACHE_FILES));
+
+    // And finally with 4th arg false
+    conf.clear();
+
+    try {
+      DistributedCache.addFileToClassPath(null, conf, fs, false);
+      fail("Accepted null archives argument");
+    } catch (NullPointerException ex) {
+      // Expected
+    }
+
+    DistributedCache.addFileToClassPath(new Path("file:///a"), conf, fs, false);
+    assertEquals("The mapreduce.job.classpath.files property was not "
+        + "set correctly", "file:/a", conf.get(MRJobConfig.CLASSPATH_FILES));
+    assertEquals("The mapreduce.job.cache.files property was not set "
+        + "correctly", "", conf.get(MRJobConfig.CACHE_FILES, ""));
+
+    DistributedCache.addFileToClassPath(new Path("file:///b"), conf, fs, false);
+    assertEquals("The mapreduce.job.classpath.files property was not "
+        + "set correctly", "file:/a,file:/b",
+        conf.get(MRJobConfig.CLASSPATH_FILES));
+    assertEquals("The mapreduce.job.cache.files property was not set "
+        + "correctly", "", conf.get(MRJobConfig.CACHE_FILES, ""));
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/605b4b61/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestLocalJobSubmission.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestLocalJobSubmission.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestLocalJobSubmission.java
index f7352f1..4a2b857 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestLocalJobSubmission.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestLocalJobSubmission.java
@@ -18,23 +18,20 @@
 package org.apache.hadoop.mapred;
 
 import java.io.File;
-import java.io.FileNotFoundException;
 import java.io.FileOutputStream;
 import java.io.IOException;
-import java.net.URL;
 import java.util.jar.JarOutputStream;
 import java.util.zip.ZipEntry;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.mapreduce.Job;
 
 import org.apache.hadoop.mapreduce.MRConfig;
 import org.apache.hadoop.mapreduce.MRJobConfig;
 import org.apache.hadoop.mapreduce.SleepJob;
 import org.apache.hadoop.util.ToolRunner;
-import org.junit.After;
-import org.junit.Before;
 import org.junit.Test;
 
 import static org.junit.Assert.*;
@@ -47,24 +44,31 @@ public class TestLocalJobSubmission {
   private static Path TEST_ROOT_DIR =
       new Path(System.getProperty("test.build.data","/tmp"));
 
-  @Before
-  public void configure() throws Exception {
-  }
+  /**
+   * Test the local job submission options of -jt local -libjars.
+   *
+   * @throws IOException thrown if there's an error creating the JAR file
+   */
+  @Test
+  public void testLocalJobLibjarsOption() throws IOException {
+    Configuration conf = new Configuration();
 
-  @After
-  public void cleanup() {
+    testLocalJobLibjarsOption(conf);
+
+    conf.setBoolean(Job.USE_WILDCARD_FOR_LIBJARS, false);
+    testLocalJobLibjarsOption(conf);
   }
 
   /**
-   * test the local job submission options of
-   * -jt local -libjars.
-   * @throws IOException
+   * Test the local job submission options of -jt local -libjars.
+   *
+   * @param conf the {@link Configuration} to use
+   * @throws IOException thrown if there's an error creating the JAR file
    */
-  @Test
-  public void testLocalJobLibjarsOption() throws IOException {
+  private void testLocalJobLibjarsOption(Configuration conf)
+      throws IOException {
     Path jarPath = makeJar(new Path(TEST_ROOT_DIR, "test.jar"));
 
-    Configuration conf = new Configuration();
     conf.set(FileSystem.FS_DEFAULT_NAME_KEY, "hdfs://localhost:9000");
     conf.set(MRConfig.FRAMEWORK_NAME, "local");
     final String[] args = {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/605b4b61/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 900bdeb..451ec57 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
@@ -911,7 +911,8 @@ public class TestMRJobs {
     }
   }
 
-  public void _testDistributedCache(String jobJarPath) throws Exception {
+  private void testDistributedCache(String jobJarPath, boolean withWildcard)
+      throws Exception {
     if (!(new File(MiniMRYarnCluster.APPJAR)).exists()) {
       LOG.info("MRAppJar " + MiniMRYarnCluster.APPJAR
            + " not found. Not running test.");
@@ -920,7 +921,7 @@ public class TestMRJobs {
 
     // Create a temporary file of length 1.
     Path first = createTempFile("distributed.first", "x");
-    // Create two jars with a single file inside them.
+    // Create three jars with a single file inside them.
     Path second =
         makeJar(new Path(TEST_ROOT_DIR, "distributed.second.jar"), 2);
     Path third =
@@ -929,16 +930,28 @@ public class TestMRJobs {
         makeJar(new Path(TEST_ROOT_DIR, "distributed.fourth.jar"), 4);
 
     Job job = Job.getInstance(mrCluster.getConfig());
-    
+
     // Set the job jar to a new "dummy" jar so we can check that its extracted 
     // properly
     job.setJar(jobJarPath);
-    // Because the job jar is a "dummy" jar, we need to include the jar with
-    // DistributedCacheChecker or it won't be able to find it
-    Path distributedCacheCheckerJar = new Path(
-            JarFinder.getJar(DistributedCacheChecker.class));
-    job.addFileToClassPath(distributedCacheCheckerJar.makeQualified(
-            localFs.getUri(), distributedCacheCheckerJar.getParent()));
+
+    if (withWildcard) {
+      // If testing with wildcards, upload the DistributedCacheChecker into HDFS
+      // and add the directory as a wildcard.
+      Path libs = new Path("testLibs");
+      Path wildcard = remoteFs.makeQualified(new Path(libs, "*"));
+
+      remoteFs.mkdirs(libs);
+      remoteFs.copyFromLocalFile(third, libs);
+      job.addCacheFile(wildcard.toUri());
+    } else {
+      // Otherwise add the DistributedCacheChecker directly to the classpath.
+      // Because the job jar is a "dummy" jar, we need to include the jar with
+      // DistributedCacheChecker or it won't be able to find it
+      Path distributedCacheCheckerJar = new Path(
+              JarFinder.getJar(DistributedCacheChecker.class));
+      job.addFileToClassPath(localFs.makeQualified(distributedCacheCheckerJar));
+    }
     
     job.setMapperClass(DistributedCacheChecker.class);
     job.setOutputFormatClass(NullOutputFormat.class);
@@ -964,11 +977,10 @@ public class TestMRJobs {
           trackingUrl.endsWith(jobId.substring(jobId.lastIndexOf("_")) + "/"));
   }
   
-  @Test (timeout = 600000)
-  public void testDistributedCache() throws Exception {
+  private void testDistributedCache(boolean withWildcard) throws Exception {
     // Test with a local (file:///) Job Jar
     Path localJobJarPath = makeJobJarWithLib(TEST_ROOT_DIR.toUri().toString());
-    _testDistributedCache(localJobJarPath.toUri().toString());
+    testDistributedCache(localJobJarPath.toUri().toString(), withWildcard);
     
     // Test with a remote (hdfs://) Job Jar
     Path remoteJobJarPath = new Path(remoteFs.getUri().toString() + "/",
@@ -978,7 +990,17 @@ public class TestMRJobs {
     if (localJobJarFile.exists()) {     // just to make sure
         localJobJarFile.delete();
     }
-    _testDistributedCache(remoteJobJarPath.toUri().toString());
+    testDistributedCache(remoteJobJarPath.toUri().toString(), withWildcard);
+  }
+
+  @Test (timeout = 300000)
+  public void testDistributedCache() throws Exception {
+    testDistributedCache(false);
+  }
+
+  @Test (timeout = 300000)
+  public void testDistributedCacheWithWildcards() throws Exception {
+    testDistributedCache(true);
   }
 
   @Test(timeout = 120000)


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


[07/17] hadoop git commit: HADOOP-9613. [JDK8] Update jersey version to latest 1.x release.

Posted by ae...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/5d58858b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/TestAHSWebServices.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/TestAHSWebServices.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/TestAHSWebServices.java
index 71b0275..3428323 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/TestAHSWebServices.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/TestAHSWebServices.java
@@ -18,6 +18,7 @@
 
 package org.apache.hadoop.yarn.server.applicationhistoryservice.webapp;
 
+import static org.apache.hadoop.yarn.webapp.WebServicesTestUtils.assertResponseStatusCode;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
@@ -67,6 +68,7 @@ import org.apache.hadoop.yarn.server.timeline.security.TimelineACLsManager;
 import org.apache.hadoop.yarn.api.records.timeline.TimelineAbout;
 import org.apache.hadoop.yarn.util.timeline.TimelineUtils;
 import org.apache.hadoop.yarn.webapp.GenericExceptionHandler;
+import org.apache.hadoop.yarn.webapp.GuiceServletConfig;
 import org.apache.hadoop.yarn.webapp.JerseyTestBase;
 import org.apache.hadoop.yarn.webapp.WebServicesTestUtils;
 import org.apache.hadoop.yarn.webapp.util.WebAppUtils;
@@ -75,15 +77,14 @@ import org.codehaus.jettison.json.JSONException;
 import org.codehaus.jettison.json.JSONObject;
 import org.junit.AfterClass;
 import org.junit.Assert;
+import org.junit.Before;
 import org.junit.BeforeClass;
 import org.junit.Test;
 import org.junit.runner.RunWith;
 import org.junit.runners.Parameterized;
 
 import com.google.inject.Guice;
-import com.google.inject.Injector;
 import com.google.inject.Singleton;
-import com.google.inject.servlet.GuiceServletContextListener;
 import com.google.inject.servlet.ServletModule;
 import com.sun.jersey.api.client.ClientResponse;
 import com.sun.jersey.api.client.ClientResponse.Status;
@@ -132,6 +133,8 @@ public class TestAHSWebServices extends JerseyTestBase {
     historyClientService.start();
     ahsWebservice = new AHSWebServices(historyClientService, conf);
     fs = FileSystem.get(conf);
+    GuiceServletConfig.setInjector(
+        Guice.createInjector(new WebServletModule()));
   }
 
   @AfterClass
@@ -148,8 +151,7 @@ public class TestAHSWebServices extends JerseyTestBase {
     return Arrays.asList(new Object[][] { { 0 }, { 1 } });
   }
 
-  private Injector injector = Guice.createInjector(new ServletModule() {
-
+  private static class WebServletModule extends ServletModule {
     @Override
     protected void configureServlets() {
       bind(JAXBContextResolver.class);
@@ -159,7 +161,14 @@ public class TestAHSWebServices extends JerseyTestBase {
       serve("/*").with(GuiceContainer.class);
       filter("/*").through(TestSimpleAuthFilter.class);
     }
-  });
+  }
+
+  @Before
+  public void setUp() throws Exception {
+    super.setUp();
+    GuiceServletConfig.setInjector(
+        Guice.createInjector(new WebServletModule()));
+  }
 
   @Singleton
   public static class TestSimpleAuthFilter extends AuthenticationFilter {
@@ -174,14 +183,6 @@ public class TestAHSWebServices extends JerseyTestBase {
     }
   }
 
-  public class GuiceServletConfig extends GuiceServletContextListener {
-
-    @Override
-    protected Injector getInjector() {
-      return injector;
-    }
-  }
-
   private int round;
 
   public TestAHSWebServices(int round) {
@@ -203,8 +204,8 @@ public class TestAHSWebServices extends JerseyTestBase {
           .queryParam("user.name", USERS[round])
           .accept(MediaType.APPLICATION_JSON)
           .get(ClientResponse.class);
-    assertEquals("404 not found expected", Status.NOT_FOUND,
-        response.getClientResponseStatus());
+    assertResponseStatusCode("404 not found expected",
+        Status.NOT_FOUND, response.getStatusInfo());
   }
 
   @Test
@@ -221,11 +222,11 @@ public class TestAHSWebServices extends JerseyTestBase {
           .accept(MediaType.APPLICATION_JSON)
           .get(ClientResponse.class);
     if (round == 1) {
-      assertEquals(Status.FORBIDDEN, response.getClientResponseStatus());
+      assertResponseStatusCode(Status.FORBIDDEN, response.getStatusInfo());
       return;
     }
-    assertEquals("404 not found expected", Status.NOT_FOUND,
-            response.getClientResponseStatus());
+    assertResponseStatusCode("404 not found expected",
+        Status.NOT_FOUND, response.getStatusInfo());
   }
 
   @Test
@@ -245,12 +246,11 @@ public class TestAHSWebServices extends JerseyTestBase {
           .accept(MediaType.APPLICATION_JSON)
           .get(ClientResponse.class);
     if (round == 1) {
-      assertEquals(
-          Status.FORBIDDEN, response.getClientResponseStatus());
+      assertResponseStatusCode(Status.FORBIDDEN, response.getStatusInfo());
       return;
     }
-    assertEquals("404 not found expected", Status.NOT_FOUND,
-            response.getClientResponseStatus());
+    assertResponseStatusCode("404 not found expected",
+        Status.NOT_FOUND, response.getStatusInfo());
   }
 
   @Test
@@ -265,7 +265,7 @@ public class TestAHSWebServices extends JerseyTestBase {
       fail("should have thrown exception on invalid uri");
     } catch (UniformInterfaceException ue) {
       ClientResponse response = ue.getResponse();
-      assertEquals(Status.NOT_FOUND, response.getClientResponseStatus());
+      assertResponseStatusCode(Status.NOT_FOUND, response.getStatusInfo());
 
       WebServicesTestUtils.checkStringMatch(
         "error string exists and shouldn't", "", responseStr);
@@ -282,7 +282,7 @@ public class TestAHSWebServices extends JerseyTestBase {
       fail("should have thrown exception on invalid uri");
     } catch (UniformInterfaceException ue) {
       ClientResponse response = ue.getResponse();
-      assertEquals(Status.NOT_FOUND, response.getClientResponseStatus());
+      assertResponseStatusCode(Status.NOT_FOUND, response.getStatusInfo());
       WebServicesTestUtils.checkStringMatch(
         "error string exists and shouldn't", "", responseStr);
     }
@@ -300,8 +300,8 @@ public class TestAHSWebServices extends JerseyTestBase {
       fail("should have thrown exception on invalid uri");
     } catch (UniformInterfaceException ue) {
       ClientResponse response = ue.getResponse();
-      assertEquals(Status.INTERNAL_SERVER_ERROR,
-        response.getClientResponseStatus());
+      assertResponseStatusCode(Status.INTERNAL_SERVER_ERROR,
+          response.getStatusInfo());
       WebServicesTestUtils.checkStringMatch(
         "error string exists and shouldn't", "", responseStr);
     }
@@ -388,8 +388,7 @@ public class TestAHSWebServices extends JerseyTestBase {
           .queryParam("user.name", USERS[round])
           .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
     if (round == 1) {
-      assertEquals(
-          Status.FORBIDDEN, response.getClientResponseStatus());
+      assertResponseStatusCode(Status.FORBIDDEN, response.getStatusInfo());
       return;
     }
     assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
@@ -415,8 +414,7 @@ public class TestAHSWebServices extends JerseyTestBase {
           .accept(MediaType.APPLICATION_JSON)
           .get(ClientResponse.class);
     if (round == 1) {
-      assertEquals(
-          Status.FORBIDDEN, response.getClientResponseStatus());
+      assertResponseStatusCode(Status.FORBIDDEN, response.getStatusInfo());
       return;
     }
     assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
@@ -445,8 +443,7 @@ public class TestAHSWebServices extends JerseyTestBase {
           .queryParam("user.name", USERS[round])
           .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
     if (round == 1) {
-      assertEquals(
-          Status.FORBIDDEN, response.getClientResponseStatus());
+      assertResponseStatusCode(Status.FORBIDDEN, response.getStatusInfo());
       return;
     }
     assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
@@ -474,8 +471,7 @@ public class TestAHSWebServices extends JerseyTestBase {
           .accept(MediaType.APPLICATION_JSON)
           .get(ClientResponse.class);
     if (round == 1) {
-      assertEquals(
-          Status.FORBIDDEN, response.getClientResponseStatus());
+      assertResponseStatusCode(Status.FORBIDDEN, response.getStatusInfo());
       return;
     }
     assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5d58858b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/timeline/webapp/TestTimelineWebServices.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/timeline/webapp/TestTimelineWebServices.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/timeline/webapp/TestTimelineWebServices.java
index 74466a8..6fd39dc 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/timeline/webapp/TestTimelineWebServices.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/timeline/webapp/TestTimelineWebServices.java
@@ -18,6 +18,7 @@
 
 package org.apache.hadoop.yarn.server.timeline.webapp;
 
+import static org.apache.hadoop.yarn.webapp.WebServicesTestUtils.assertResponseStatusCode;
 import static org.junit.Assert.assertEquals;
 import static org.mockito.Matchers.any;
 import static org.mockito.Mockito.doNothing;
@@ -61,14 +62,14 @@ import org.apache.hadoop.yarn.server.timeline.security.TimelineAuthenticationFil
 import org.apache.hadoop.yarn.api.records.timeline.TimelineAbout;
 import org.apache.hadoop.yarn.util.timeline.TimelineUtils;
 import org.apache.hadoop.yarn.webapp.GenericExceptionHandler;
+import org.apache.hadoop.yarn.webapp.GuiceServletConfig;
 import org.apache.hadoop.yarn.webapp.JerseyTestBase;
 import org.apache.hadoop.yarn.webapp.YarnJacksonJaxbJsonProvider;
 import org.junit.Assert;
+import org.junit.Before;
 import org.junit.Test;
 
 import com.google.inject.Guice;
-import com.google.inject.Injector;
-import com.google.inject.servlet.GuiceServletContextListener;
 import com.google.inject.servlet.ServletModule;
 import com.sun.jersey.api.client.ClientResponse;
 import com.sun.jersey.api.client.WebResource;
@@ -81,10 +82,9 @@ public class TestTimelineWebServices extends JerseyTestBase {
   private static TimelineStore store;
   private static TimelineACLsManager timelineACLsManager;
   private static AdminACLsManager adminACLsManager;
-  private long beforeTime;
-
-  private Injector injector = Guice.createInjector(new ServletModule() {
+  private static long beforeTime;
 
+  private static class WebServletModule extends ServletModule {
     @SuppressWarnings("unchecked")
     @Override
     protected void configureServlets() {
@@ -129,7 +129,7 @@ public class TestTimelineWebServices extends JerseyTestBase {
       when(filterConfig.getInitParameterNames()).thenReturn(names);
       when(filterConfig.getInitParameter(
           DelegationTokenAuthenticationHandler.TOKEN_KIND)).thenReturn(
-              TimelineDelegationTokenIdentifier.KIND_NAME.toString());
+          TimelineDelegationTokenIdentifier.KIND_NAME.toString());
       try {
         taFilter.init(filterConfig);
       } catch (ServletException e) {
@@ -146,18 +146,21 @@ public class TestTimelineWebServices extends JerseyTestBase {
       }
       filter("/*").through(taFilter);
     }
+  }
 
-  });
-
-  public class GuiceServletConfig extends GuiceServletContextListener {
+  static {
+    GuiceServletConfig.setInjector(
+        Guice.createInjector(new WebServletModule()));
+  }
 
-    @Override
-    protected Injector getInjector() {
-      return injector;
-    }
+  @Before
+  public void setUp() throws Exception {
+    super.setUp();
+    GuiceServletConfig.setInjector(
+        Guice.createInjector(new WebServletModule()));
   }
 
-  private TimelineStore mockTimelineStore()
+  private static TimelineStore mockTimelineStore()
       throws Exception {
     beforeTime = System.currentTimeMillis() - 1;
     TestMemoryTimelineStore store =
@@ -477,8 +480,7 @@ public class TestTimelineWebServices extends JerseyTestBase {
         .type(MediaType.APPLICATION_JSON)
         .post(ClientResponse.class, entities);
     assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
-    assertEquals(ClientResponse.Status.FORBIDDEN,
-        response.getClientResponseStatus());
+    assertResponseStatusCode(Status.FORBIDDEN, response.getStatusInfo());
 
     response = r.path("ws").path("v1").path("timeline")
         .queryParam("user.name", "tester")
@@ -519,8 +521,7 @@ public class TestTimelineWebServices extends JerseyTestBase {
         .queryParam("user.name", "tester").accept(MediaType.APPLICATION_JSON)
          .type(MediaType.APPLICATION_JSON).post(ClientResponse.class, entities);
     assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
-    assertEquals(ClientResponse.Status.BAD_REQUEST,
-        response.getClientResponseStatus());
+    assertResponseStatusCode(Status.BAD_REQUEST, response.getStatusInfo());
   }
 
   @Test
@@ -703,8 +704,7 @@ public class TestTimelineWebServices extends JerseyTestBase {
           .accept(MediaType.APPLICATION_JSON)
           .get(ClientResponse.class);
       assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
-      assertEquals(ClientResponse.Status.NOT_FOUND,
-          response.getClientResponseStatus());
+      assertResponseStatusCode(Status.NOT_FOUND, response.getStatusInfo());
     } finally {
       timelineACLsManager.setAdminACLsManager(oldAdminACLsManager);
     }
@@ -869,8 +869,7 @@ public class TestTimelineWebServices extends JerseyTestBase {
           .accept(MediaType.APPLICATION_JSON)
           .get(ClientResponse.class);
       Assert.assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
-      Assert.assertEquals(ClientResponse.Status.NOT_FOUND,
-          response.getClientResponseStatus());
+      assertResponseStatusCode(Status.NOT_FOUND, response.getStatusInfo());
     } finally {
       timelineACLsManager.setAdminACLsManager(oldAdminACLsManager);
     }
@@ -938,8 +937,7 @@ public class TestTimelineWebServices extends JerseyTestBase {
         .type(MediaType.APPLICATION_JSON)
         .put(ClientResponse.class, domain);
     assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
-    assertEquals(ClientResponse.Status.FORBIDDEN,
-        response.getClientResponseStatus());
+    assertResponseStatusCode(Status.FORBIDDEN, response.getStatusInfo());
 
     response = r.path("ws").path("v1")
         .path("timeline").path("domain")
@@ -947,8 +945,8 @@ public class TestTimelineWebServices extends JerseyTestBase {
         .accept(MediaType.APPLICATION_JSON)
         .type(MediaType.APPLICATION_JSON)
         .put(ClientResponse.class, domain);
-    assertEquals(Status.OK.getStatusCode(), response.getStatus());
-    
+    assertResponseStatusCode(Status.OK, response.getStatusInfo());
+
     // Verify the domain exists
     response = r.path("ws").path("v1").path("timeline")
         .path("domain").path("test_domain_id")
@@ -969,7 +967,7 @@ public class TestTimelineWebServices extends JerseyTestBase {
         .accept(MediaType.APPLICATION_JSON)
         .type(MediaType.APPLICATION_JSON)
         .put(ClientResponse.class, domain);
-    assertEquals(Status.OK.getStatusCode(), response.getStatus());
+    assertResponseStatusCode(Status.OK, response.getStatusInfo());
 
     // Verify the domain is updated
     response = r.path("ws").path("v1").path("timeline")
@@ -997,7 +995,7 @@ public class TestTimelineWebServices extends JerseyTestBase {
           .accept(MediaType.APPLICATION_JSON)
           .type(MediaType.APPLICATION_JSON)
           .put(ClientResponse.class, domain);
-      assertEquals(Status.OK.getStatusCode(), response.getStatus());
+      assertResponseStatusCode(Status.OK, response.getStatusInfo());
 
       // Update the domain by another user
       response = r.path("ws").path("v1")
@@ -1006,7 +1004,7 @@ public class TestTimelineWebServices extends JerseyTestBase {
           .accept(MediaType.APPLICATION_JSON)
           .type(MediaType.APPLICATION_JSON)
           .put(ClientResponse.class, domain);
-      assertEquals(Status.FORBIDDEN.getStatusCode(), response.getStatus());
+      assertResponseStatusCode(Status.FORBIDDEN, response.getStatusInfo());
     } finally {
       timelineACLsManager.setAdminACLsManager(oldAdminACLsManager);
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5d58858b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/webapp/TestNMWebServices.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/TestNMWebServices.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/webapp/TestNMWebServices.java
index a4305da..75619f1 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/webapp/TestNMWebServices.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/webapp/TestNMWebServices.java
@@ -18,6 +18,7 @@
 
 package org.apache.hadoop.yarn.server.nodemanager.webapp;
 
+import static org.apache.hadoop.yarn.webapp.WebServicesTestUtils.assertResponseStatusCode;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
@@ -30,6 +31,7 @@ import javax.ws.rs.core.MediaType;
 import javax.xml.parsers.DocumentBuilder;
 import javax.xml.parsers.DocumentBuilderFactory;
 
+import org.apache.hadoop.yarn.webapp.GuiceServletConfig;
 import org.junit.Assert;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileUtil;
@@ -69,8 +71,6 @@ import org.w3c.dom.NodeList;
 import org.xml.sax.InputSource;
 
 import com.google.inject.Guice;
-import com.google.inject.Injector;
-import com.google.inject.servlet.GuiceServletContextListener;
 import com.google.inject.servlet.ServletModule;
 import com.sun.jersey.api.client.ClientResponse;
 import com.sun.jersey.api.client.ClientResponse.Status;
@@ -95,7 +95,7 @@ public class TestNMWebServices extends JerseyTestBase {
   private static File testLogDir = new File("target",
       TestNMWebServices.class.getSimpleName() + "LogDir");
 
-  private Injector injector = Guice.createInjector(new ServletModule() {
+  private static class WebServletModule extends ServletModule {
 
     @Override
     protected void configureServlets() {
@@ -148,14 +148,11 @@ public class TestNMWebServices extends JerseyTestBase {
 
       serve("/*").with(GuiceContainer.class);
     }
-  });
+  };
 
-  public class GuiceServletConfig extends GuiceServletContextListener {
-
-    @Override
-    protected Injector getInjector() {
-      return injector;
-    }
+  static {
+    GuiceServletConfig.setInjector(
+        Guice.createInjector(new WebServletModule()));
   }
 
   @Before
@@ -164,6 +161,8 @@ public class TestNMWebServices extends JerseyTestBase {
     super.setUp();
     testRootDir.mkdirs();
     testLogDir.mkdir();
+    GuiceServletConfig.setInjector(
+        Guice.createInjector(new WebServletModule()));
   }
 
   @AfterClass
@@ -190,7 +189,7 @@ public class TestNMWebServices extends JerseyTestBase {
       fail("should have thrown exception on invalid uri");
     } catch (UniformInterfaceException ue) {
       ClientResponse response = ue.getResponse();
-      assertEquals(Status.NOT_FOUND, response.getClientResponseStatus());
+      assertResponseStatusCode(Status.NOT_FOUND, response.getStatusInfo());
       WebServicesTestUtils.checkStringMatch(
           "error string exists and shouldn't", "", responseStr);
     }
@@ -206,8 +205,8 @@ public class TestNMWebServices extends JerseyTestBase {
       fail("should have thrown exception on invalid uri");
     } catch (UniformInterfaceException ue) {
       ClientResponse response = ue.getResponse();
-      assertEquals(Status.INTERNAL_SERVER_ERROR,
-          response.getClientResponseStatus());
+      assertResponseStatusCode(Status.INTERNAL_SERVER_ERROR,
+          response.getStatusInfo());
       WebServicesTestUtils.checkStringMatch(
           "error string exists and shouldn't", "", responseStr);
     }
@@ -222,7 +221,7 @@ public class TestNMWebServices extends JerseyTestBase {
       fail("should have thrown exception on invalid uri");
     } catch (UniformInterfaceException ue) {
       ClientResponse response = ue.getResponse();
-      assertEquals(Status.NOT_FOUND, response.getClientResponseStatus());
+      assertResponseStatusCode(Status.NOT_FOUND, response.getStatusInfo());
       WebServicesTestUtils.checkStringMatch(
           "error string exists and shouldn't", "", responseStr);
     }
@@ -418,7 +417,7 @@ public class TestNMWebServices extends JerseyTestBase {
     response = r.path("ws").path("v1").path("node")
         .path("containerlogs").path(containerIdStr).path("uhhh")
         .accept(MediaType.TEXT_PLAIN).get(ClientResponse.class);
-    Assert.assertEquals(Status.NOT_FOUND.getStatusCode(), response.getStatus());
+    assertResponseStatusCode(Status.NOT_FOUND, response.getStatusInfo());
     responseText = response.getEntity(String.class);
     assertTrue(responseText.contains("Cannot find this log on the local disk."));
     

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5d58858b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/webapp/TestNMWebServicesApps.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/TestNMWebServicesApps.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/webapp/TestNMWebServicesApps.java
index b95b180..1c34b44 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/webapp/TestNMWebServicesApps.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/webapp/TestNMWebServicesApps.java
@@ -18,6 +18,7 @@
 
 package org.apache.hadoop.yarn.server.nodemanager.webapp;
 
+import static org.apache.hadoop.yarn.webapp.WebServicesTestUtils.assertResponseStatusCode;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
@@ -33,7 +34,6 @@ import javax.xml.parsers.DocumentBuilderFactory;
 
 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.NodeId;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
@@ -51,6 +51,7 @@ import org.apache.hadoop.yarn.server.nodemanager.webapp.WebServer.NMWebApp;
 import org.apache.hadoop.yarn.server.security.ApplicationACLsManager;
 import org.apache.hadoop.yarn.server.utils.BuilderUtils;
 import org.apache.hadoop.yarn.webapp.GenericExceptionHandler;
+import org.apache.hadoop.yarn.webapp.GuiceServletConfig;
 import org.apache.hadoop.yarn.webapp.JerseyTestBase;
 import org.apache.hadoop.yarn.webapp.WebApp;
 import org.apache.hadoop.yarn.webapp.WebServicesTestUtils;
@@ -67,8 +68,6 @@ import org.w3c.dom.NodeList;
 import org.xml.sax.InputSource;
 
 import com.google.inject.Guice;
-import com.google.inject.Injector;
-import com.google.inject.servlet.GuiceServletContextListener;
 import com.google.inject.servlet.ServletModule;
 import com.sun.jersey.api.client.ClientResponse;
 import com.sun.jersey.api.client.ClientResponse.Status;
@@ -91,8 +90,7 @@ public class TestNMWebServicesApps extends JerseyTestBase {
   private static File testLogDir = new File("target",
       TestNMWebServicesApps.class.getSimpleName() + "LogDir");
 
-  private Injector injector = Guice.createInjector(new ServletModule() {
-
+  private static class WebServletModule extends ServletModule {
     @Override
     protected void configureServlets() {
       conf.set(YarnConfiguration.NM_LOCAL_DIRS, testRootDir.getAbsolutePath());
@@ -148,20 +146,19 @@ public class TestNMWebServicesApps extends JerseyTestBase {
 
       serve("/*").with(GuiceContainer.class);
     }
-  });
-
-  public class GuiceServletConfig extends GuiceServletContextListener {
+  }
 
-    @Override
-    protected Injector getInjector() {
-      return injector;
-    }
+  static {
+    GuiceServletConfig.setInjector(
+        Guice.createInjector(new WebServletModule()));
   }
 
   @Before
   @Override
   public void setUp() throws Exception {
     super.setUp();
+    GuiceServletConfig.setInjector(
+        Guice.createInjector(new WebServletModule()));
     testRootDir.mkdirs();
     testLogDir.mkdir();
   }
@@ -187,7 +184,8 @@ public class TestNMWebServicesApps extends JerseyTestBase {
         .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
     assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
     JSONObject json = response.getEntity(JSONObject.class);
-    assertEquals("apps isn't NULL", JSONObject.NULL, json.get("apps"));
+    assertEquals("apps isn't empty",
+        new JSONObject().toString(), json.get("apps").toString());
   }
 
   private HashMap<String, String> addAppContainers(Application app) 
@@ -297,7 +295,8 @@ public class TestNMWebServicesApps extends JerseyTestBase {
         .get(ClientResponse.class);
     assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
     JSONObject json = response.getEntity(JSONObject.class);
-    assertEquals("apps is not null", JSONObject.NULL, json.get("apps"));
+    assertEquals("apps is not empty",
+        new JSONObject().toString(), json.get("apps").toString());
   }
 
   @Test
@@ -317,7 +316,7 @@ public class TestNMWebServicesApps extends JerseyTestBase {
     } catch (UniformInterfaceException ue) {
       ClientResponse response = ue.getResponse();
 
-      assertEquals(Status.BAD_REQUEST, response.getClientResponseStatus());
+      assertResponseStatusCode(Status.BAD_REQUEST, response.getStatusInfo());
       assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
       JSONObject msg = response.getEntity(JSONObject.class);
       JSONObject exception = msg.getJSONObject("RemoteException");
@@ -379,7 +378,8 @@ public class TestNMWebServicesApps extends JerseyTestBase {
     assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
     JSONObject json = response.getEntity(JSONObject.class);
 
-    assertEquals("apps is not null", JSONObject.NULL, json.get("apps"));
+    assertEquals("apps is not empty",
+        new JSONObject().toString(), json.get("apps").toString());
   }
 
   @Test
@@ -400,7 +400,7 @@ public class TestNMWebServicesApps extends JerseyTestBase {
     } catch (UniformInterfaceException ue) {
       ClientResponse response = ue.getResponse();
 
-      assertEquals(Status.BAD_REQUEST, response.getClientResponseStatus());
+      assertResponseStatusCode(Status.BAD_REQUEST, response.getStatusInfo());
       assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
       JSONObject msg = response.getEntity(JSONObject.class);
       JSONObject exception = msg.getJSONObject("RemoteException");
@@ -430,7 +430,7 @@ public class TestNMWebServicesApps extends JerseyTestBase {
     } catch (UniformInterfaceException ue) {
       ClientResponse response = ue.getResponse();
 
-      assertEquals(Status.BAD_REQUEST, response.getClientResponseStatus());
+      assertResponseStatusCode(Status.BAD_REQUEST, response.getStatusInfo());
       assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
       JSONObject msg = response.getEntity(JSONObject.class);
       JSONObject exception = msg.getJSONObject("RemoteException");
@@ -461,7 +461,7 @@ public class TestNMWebServicesApps extends JerseyTestBase {
     } catch (UniformInterfaceException ue) {
       ClientResponse response = ue.getResponse();
 
-      assertEquals(Status.BAD_REQUEST, response.getClientResponseStatus());
+      assertResponseStatusCode(Status.BAD_REQUEST, response.getStatusInfo());
       assertEquals(MediaType.APPLICATION_XML_TYPE, response.getType());
       String msg = response.getEntity(String.class);
 
@@ -556,7 +556,7 @@ public class TestNMWebServicesApps extends JerseyTestBase {
       fail("should have thrown exception on invalid user query");
     } catch (UniformInterfaceException ue) {
       ClientResponse response = ue.getResponse();
-      assertEquals(Status.BAD_REQUEST, response.getClientResponseStatus());
+      assertResponseStatusCode(Status.BAD_REQUEST, response.getStatusInfo());
       assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
       JSONObject msg = response.getEntity(JSONObject.class);
       JSONObject exception = msg.getJSONObject("RemoteException");
@@ -593,7 +593,7 @@ public class TestNMWebServicesApps extends JerseyTestBase {
       fail("should have thrown exception on invalid user query");
     } catch (UniformInterfaceException ue) {
       ClientResponse response = ue.getResponse();
-      assertEquals(Status.NOT_FOUND, response.getClientResponseStatus());
+      assertResponseStatusCode(Status.NOT_FOUND, response.getStatusInfo());
       assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
       JSONObject msg = response.getEntity(JSONObject.class);
       JSONObject exception = msg.getJSONObject("RemoteException");

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5d58858b/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 7ec8f27..bec448d 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
@@ -18,6 +18,7 @@
 
 package org.apache.hadoop.yarn.server.nodemanager.webapp;
 
+import static org.apache.hadoop.yarn.webapp.WebServicesTestUtils.assertResponseStatusCode;
 import static org.apache.hadoop.yarn.util.StringHelper.ujoin;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
@@ -38,7 +39,6 @@ import javax.xml.parsers.DocumentBuilderFactory;
 import com.sun.jersey.api.client.filter.LoggingFilter;
 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;
@@ -58,6 +58,7 @@ import org.apache.hadoop.yarn.server.security.ApplicationACLsManager;
 import org.apache.hadoop.yarn.server.utils.BuilderUtils;
 import org.apache.hadoop.yarn.util.ConverterUtils;
 import org.apache.hadoop.yarn.webapp.GenericExceptionHandler;
+import org.apache.hadoop.yarn.webapp.GuiceServletConfig;
 import org.apache.hadoop.yarn.webapp.JerseyTestBase;
 import org.apache.hadoop.yarn.webapp.WebApp;
 import org.apache.hadoop.yarn.webapp.WebServicesTestUtils;
@@ -73,8 +74,6 @@ import org.w3c.dom.NodeList;
 import org.xml.sax.InputSource;
 
 import com.google.inject.Guice;
-import com.google.inject.Injector;
-import com.google.inject.servlet.GuiceServletContextListener;
 import com.google.inject.servlet.ServletModule;
 import com.sun.jersey.api.client.ClientResponse;
 import com.sun.jersey.api.client.ClientResponse.Status;
@@ -97,7 +96,7 @@ public class TestNMWebServicesContainers extends JerseyTestBase {
   private static File testLogDir = new File("target",
       TestNMWebServicesContainers.class.getSimpleName() + "LogDir");
 
-  private Injector injector = Guice.createInjector(new ServletModule() {
+  private static class WebServletModule extends ServletModule {
     @Override
     protected void configureServlets() {
       resourceView = new ResourceView() {
@@ -158,20 +157,19 @@ public class TestNMWebServicesContainers extends JerseyTestBase {
 
       serve("/*").with(GuiceContainer.class);
     }
-  });
-
-  public class GuiceServletConfig extends GuiceServletContextListener {
+  }
 
-    @Override
-    protected Injector getInjector() {
-      return injector;
-    }
+  static {
+    GuiceServletConfig.setInjector(
+        Guice.createInjector(new WebServletModule()));
   }
 
   @Before
   @Override
   public void setUp() throws Exception {
     super.setUp();
+    GuiceServletConfig.setInjector(
+        Guice.createInjector(new WebServletModule()));
     testRootDir.mkdirs();
     testLogDir.mkdir();
   }
@@ -198,7 +196,8 @@ public class TestNMWebServicesContainers extends JerseyTestBase {
         .get(ClientResponse.class);
     assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
     JSONObject json = response.getEntity(JSONObject.class);
-    assertEquals("apps isn't NULL", JSONObject.NULL, json.get("containers"));
+    assertEquals("apps isn't empty",
+        new JSONObject().toString(), json.get("containers").toString());
   }
 
   private HashMap<String, String> addAppContainers(Application app) 
@@ -337,7 +336,7 @@ public class TestNMWebServicesContainers extends JerseyTestBase {
       fail("should have thrown exception on invalid user query");
     } catch (UniformInterfaceException ue) {
       ClientResponse response = ue.getResponse();
-      assertEquals(Status.BAD_REQUEST, response.getClientResponseStatus());
+      assertResponseStatusCode(Status.BAD_REQUEST, response.getStatusInfo());
       assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
       JSONObject msg = response.getEntity(JSONObject.class);
       JSONObject exception = msg.getJSONObject("RemoteException");
@@ -371,7 +370,7 @@ public class TestNMWebServicesContainers extends JerseyTestBase {
       fail("should have thrown exception on invalid user query");
     } catch (UniformInterfaceException ue) {
       ClientResponse response = ue.getResponse();
-      assertEquals(Status.BAD_REQUEST, response.getClientResponseStatus());
+      assertResponseStatusCode(Status.BAD_REQUEST, response.getStatusInfo());
       assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
       JSONObject msg = response.getEntity(JSONObject.class);
       JSONObject exception = msg.getJSONObject("RemoteException");
@@ -405,7 +404,7 @@ public class TestNMWebServicesContainers extends JerseyTestBase {
       fail("should have thrown exception on invalid user query");
     } catch (UniformInterfaceException ue) {
       ClientResponse response = ue.getResponse();
-      assertEquals(Status.NOT_FOUND, response.getClientResponseStatus());
+      assertResponseStatusCode(Status.NOT_FOUND, response.getStatusInfo());
       assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
       JSONObject msg = response.getEntity(JSONObject.class);
       JSONObject exception = msg.getJSONObject("RemoteException");

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5d58858b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServices.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/TestRMWebServices.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServices.java
index 50a4849..082d043 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServices.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServices.java
@@ -18,6 +18,7 @@
 
 package org.apache.hadoop.yarn.server.resourcemanager.webapp;
 
+import static org.apache.hadoop.yarn.webapp.WebServicesTestUtils.assertResponseStatusCode;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
@@ -41,9 +42,7 @@ import javax.xml.parsers.DocumentBuilderFactory;
 
 import org.apache.commons.io.FileUtils;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.io.TwoDArrayWritable;
 import org.apache.hadoop.service.Service.STATE;
-import org.apache.hadoop.test.GenericTestUtils.SleepAnswer;
 import org.apache.hadoop.util.VersionInfo;
 import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationsRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationsResponse;
@@ -68,6 +67,7 @@ import org.apache.hadoop.yarn.util.AdHocLogDumper;
 import org.apache.hadoop.yarn.util.YarnVersionInfo;
 import org.apache.hadoop.yarn.webapp.ForbiddenException;
 import org.apache.hadoop.yarn.webapp.GenericExceptionHandler;
+import org.apache.hadoop.yarn.webapp.GuiceServletConfig;
 import org.apache.hadoop.yarn.webapp.JerseyTestBase;
 import org.apache.hadoop.yarn.webapp.WebServicesTestUtils;
 import org.codehaus.jettison.json.JSONException;
@@ -81,8 +81,6 @@ import org.w3c.dom.NodeList;
 import org.xml.sax.InputSource;
 
 import com.google.inject.Guice;
-import com.google.inject.Injector;
-import com.google.inject.servlet.GuiceServletContextListener;
 import com.google.inject.servlet.ServletModule;
 import com.sun.jersey.api.client.ClientResponse;
 import com.sun.jersey.api.client.ClientResponse.Status;
@@ -95,7 +93,7 @@ public class TestRMWebServices extends JerseyTestBase {
 
   private static MockRM rm;
 
-  private Injector injector = Guice.createInjector(new ServletModule() {
+  private static class WebServletModule extends ServletModule {
     @Override
     protected void configureServlets() {
       bind(JAXBContextResolver.class);
@@ -108,20 +106,19 @@ public class TestRMWebServices extends JerseyTestBase {
       bind(ResourceManager.class).toInstance(rm);
       serve("/*").with(GuiceContainer.class);
     }
-  });
-
-  public class GuiceServletConfig extends GuiceServletContextListener {
+  }
 
-    @Override
-    protected Injector getInjector() {
-      return injector;
-    }
+  static {
+    GuiceServletConfig.setInjector(
+        Guice.createInjector(new WebServletModule()));
   }
 
   @Before
   @Override
   public void setUp() throws Exception {
     super.setUp();
+    GuiceServletConfig.setInjector(
+        Guice.createInjector(new WebServletModule()));
   }
 
   public TestRMWebServices() {
@@ -162,7 +159,7 @@ public class TestRMWebServices extends JerseyTestBase {
       fail("should have thrown exception on invalid uri");
     } catch (UniformInterfaceException ue) {
       ClientResponse response = ue.getResponse();
-      assertEquals(Status.NOT_FOUND, response.getClientResponseStatus());
+      assertResponseStatusCode(Status.NOT_FOUND, response.getStatusInfo());
 
       WebServicesTestUtils.checkStringMatch(
           "error string exists and shouldn't", "", responseStr);
@@ -178,7 +175,7 @@ public class TestRMWebServices extends JerseyTestBase {
       fail("should have thrown exception on invalid uri");
     } catch (UniformInterfaceException ue) {
       ClientResponse response = ue.getResponse();
-      assertEquals(Status.NOT_FOUND, response.getClientResponseStatus());
+      assertResponseStatusCode(Status.NOT_FOUND, response.getStatusInfo());
       WebServicesTestUtils.checkStringMatch(
           "error string exists and shouldn't", "", responseStr);
     }
@@ -194,8 +191,8 @@ public class TestRMWebServices extends JerseyTestBase {
       fail("should have thrown exception on invalid uri");
     } catch (UniformInterfaceException ue) {
       ClientResponse response = ue.getResponse();
-      assertEquals(Status.INTERNAL_SERVER_ERROR,
-          response.getClientResponseStatus());
+      assertResponseStatusCode(Status.INTERNAL_SERVER_ERROR,
+          response.getStatusInfo());
       WebServicesTestUtils.checkStringMatch(
           "error string exists and shouldn't", "", responseStr);
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5d58858b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesApps.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesApps.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesApps.java
index 9ba5233..4f99c5c 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesApps.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesApps.java
@@ -18,6 +18,7 @@
 
 package org.apache.hadoop.yarn.server.resourcemanager.webapp;
 
+import static org.apache.hadoop.yarn.webapp.WebServicesTestUtils.assertResponseStatusCode;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
@@ -46,6 +47,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptS
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fifo.FifoScheduler;
 import org.apache.hadoop.yarn.webapp.GenericExceptionHandler;
+import org.apache.hadoop.yarn.webapp.GuiceServletConfig;
 import org.apache.hadoop.yarn.webapp.JerseyTestBase;
 import org.apache.hadoop.yarn.webapp.WebServicesTestUtils;
 import org.codehaus.jettison.json.JSONArray;
@@ -59,8 +61,6 @@ import org.w3c.dom.NodeList;
 import org.xml.sax.InputSource;
 
 import com.google.inject.Guice;
-import com.google.inject.Injector;
-import com.google.inject.servlet.GuiceServletContextListener;
 import com.google.inject.servlet.ServletModule;
 import com.sun.jersey.api.client.ClientResponse;
 import com.sun.jersey.api.client.ClientResponse.Status;
@@ -76,7 +76,7 @@ public class TestRMWebServicesApps extends JerseyTestBase {
   
   private static final int CONTAINER_MB = 1024;
 
-  private Injector injector = Guice.createInjector(new ServletModule() {
+  private static class WebServletModule extends ServletModule {
     @Override
     protected void configureServlets() {
       bind(JAXBContextResolver.class);
@@ -91,20 +91,19 @@ public class TestRMWebServicesApps extends JerseyTestBase {
       bind(ResourceManager.class).toInstance(rm);
       serve("/*").with(GuiceContainer.class);
     }
-  });
-
-  public class GuiceServletConfig extends GuiceServletContextListener {
+  }
 
-    @Override
-    protected Injector getInjector() {
-      return injector;
-    }
+  static {
+    GuiceServletConfig.setInjector(
+        Guice.createInjector(new WebServletModule()));
   }
 
   @Before
   @Override
   public void setUp() throws Exception {
     super.setUp();
+    GuiceServletConfig.setInjector(
+        Guice.createInjector(new WebServletModule()));
   }
 
   public TestRMWebServicesApps() {
@@ -350,7 +349,8 @@ public class TestRMWebServicesApps extends JerseyTestBase {
     assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
     JSONObject json = response.getEntity(JSONObject.class);
     assertEquals("incorrect number of elements", 1, json.length());
-    assertEquals("apps is not null", JSONObject.NULL, json.get("apps"));
+    assertEquals("apps is not empty",
+        new JSONObject().toString(), json.get("apps").toString());
     rm.stop();
   }
 
@@ -369,7 +369,8 @@ public class TestRMWebServicesApps extends JerseyTestBase {
     assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
     JSONObject json = response.getEntity(JSONObject.class);
     assertEquals("incorrect number of elements", 1, json.length());
-    assertEquals("apps is not null", JSONObject.NULL, json.get("apps"));
+    assertEquals("apps is not empty",
+        new JSONObject().toString(), json.get("apps").toString());
     rm.stop();
   }
 
@@ -388,7 +389,7 @@ public class TestRMWebServicesApps extends JerseyTestBase {
       fail("should have thrown exception on invalid state query");
     } catch (UniformInterfaceException ue) {
       ClientResponse response = ue.getResponse();
-      assertEquals(Status.BAD_REQUEST, response.getClientResponseStatus());
+      assertResponseStatusCode(Status.BAD_REQUEST, response.getStatusInfo());
       assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
       JSONObject msg = response.getEntity(JSONObject.class);
       JSONObject exception = msg.getJSONObject("RemoteException");
@@ -425,7 +426,7 @@ public class TestRMWebServicesApps extends JerseyTestBase {
       fail("should have thrown exception on invalid state query");
     } catch (UniformInterfaceException ue) {
       ClientResponse response = ue.getResponse();
-      assertEquals(Status.BAD_REQUEST, response.getClientResponseStatus());
+      assertResponseStatusCode(Status.BAD_REQUEST, response.getStatusInfo());
       assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
       JSONObject msg = response.getEntity(JSONObject.class);
       JSONObject exception = msg.getJSONObject("RemoteException");
@@ -484,7 +485,8 @@ public class TestRMWebServicesApps extends JerseyTestBase {
     assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
     JSONObject json = response.getEntity(JSONObject.class);
     assertEquals("incorrect number of elements", 1, json.length());
-    assertEquals("apps is not null", JSONObject.NULL, json.get("apps"));
+    assertEquals("apps is not null",
+        new JSONObject().toString(), json.get("apps").toString());
     rm.stop();
   }
 
@@ -503,7 +505,7 @@ public class TestRMWebServicesApps extends JerseyTestBase {
       fail("should have thrown exception on invalid state query");
     } catch (UniformInterfaceException ue) {
       ClientResponse response = ue.getResponse();
-      assertEquals(Status.BAD_REQUEST, response.getClientResponseStatus());
+      assertResponseStatusCode(Status.BAD_REQUEST, response.getStatusInfo());
       assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
       JSONObject msg = response.getEntity(JSONObject.class);
       JSONObject exception = msg.getJSONObject("RemoteException");
@@ -660,7 +662,8 @@ public class TestRMWebServicesApps extends JerseyTestBase {
     assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
     JSONObject json = response.getEntity(JSONObject.class);
     assertEquals("incorrect number of elements", 1, json.length());
-    assertEquals("apps is not null", JSONObject.NULL, json.get("apps"));
+    assertEquals("apps is not empty",
+        new JSONObject().toString(), json.get("apps").toString());
     rm.stop();
   }
 
@@ -1070,7 +1073,7 @@ public class TestRMWebServicesApps extends JerseyTestBase {
           .path("appstatistics")
           .queryParam("applicationTypes", "MAPREDUCE,OTHER")
           .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
-      assertEquals(Status.BAD_REQUEST, response.getClientResponseStatus());
+      assertResponseStatusCode(Status.BAD_REQUEST, response.getStatusInfo());
       assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
       json = response.getEntity(JSONObject.class);
       assertEquals("incorrect number of elements", 1, json.length());
@@ -1117,7 +1120,7 @@ public class TestRMWebServicesApps extends JerseyTestBase {
       response = r.path("ws").path("v1").path("cluster")
           .path("appstatistics").queryParam("states", "wrong_state")
           .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
-      assertEquals(Status.BAD_REQUEST, response.getClientResponseStatus());
+      assertResponseStatusCode(Status.BAD_REQUEST, response.getStatusInfo());
       assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
       json = response.getEntity(JSONObject.class);
       assertEquals("incorrect number of elements", 1, json.length());
@@ -1185,7 +1188,7 @@ public class TestRMWebServicesApps extends JerseyTestBase {
     } catch (UniformInterfaceException ue) {
       ClientResponse response = ue.getResponse();
 
-      assertEquals(Status.BAD_REQUEST, response.getClientResponseStatus());
+      assertResponseStatusCode(Status.BAD_REQUEST, response.getStatusInfo());
       assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
       JSONObject msg = response.getEntity(JSONObject.class);
       JSONObject exception = msg.getJSONObject("RemoteException");
@@ -1223,7 +1226,7 @@ public class TestRMWebServicesApps extends JerseyTestBase {
     } catch (UniformInterfaceException ue) {
       ClientResponse response = ue.getResponse();
 
-      assertEquals(Status.NOT_FOUND, response.getClientResponseStatus());
+      assertResponseStatusCode(Status.NOT_FOUND, response.getStatusInfo());
       assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
 
       JSONObject msg = response.getEntity(JSONObject.class);
@@ -1515,7 +1518,7 @@ public class TestRMWebServicesApps extends JerseyTestBase {
     } catch (UniformInterfaceException ue) {
       ClientResponse response = ue.getResponse();
 
-      assertEquals(Status.BAD_REQUEST, response.getClientResponseStatus());
+      assertResponseStatusCode(Status.BAD_REQUEST, response.getStatusInfo());
       assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
       JSONObject msg = response.getEntity(JSONObject.class);
       JSONObject exception = msg.getJSONObject("RemoteException");
@@ -1554,7 +1557,7 @@ public class TestRMWebServicesApps extends JerseyTestBase {
     } catch (UniformInterfaceException ue) {
       ClientResponse response = ue.getResponse();
 
-      assertEquals(Status.BAD_REQUEST, response.getClientResponseStatus());
+      assertResponseStatusCode(Status.BAD_REQUEST, response.getStatusInfo());
       assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
       JSONObject msg = response.getEntity(JSONObject.class);
       JSONObject exception = msg.getJSONObject("RemoteException");
@@ -1592,7 +1595,7 @@ public class TestRMWebServicesApps extends JerseyTestBase {
     } catch (UniformInterfaceException ue) {
       ClientResponse response = ue.getResponse();
 
-      assertEquals(Status.NOT_FOUND, response.getClientResponseStatus());
+      assertResponseStatusCode(Status.NOT_FOUND, response.getStatusInfo());
       assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
 
       JSONObject msg = response.getEntity(JSONObject.class);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5d58858b/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 682ed75..53ef031 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
@@ -18,6 +18,7 @@
 
 package org.apache.hadoop.yarn.server.resourcemanager.webapp;
 
+import static org.apache.hadoop.yarn.webapp.WebServicesTestUtils.assertResponseStatusCode;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assume.assumeTrue;
@@ -87,6 +88,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.LocalResourceInf
 import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.LogAggregationContextInfo;
 import org.apache.hadoop.yarn.util.ConverterUtils;
 import org.apache.hadoop.yarn.webapp.GenericExceptionHandler;
+import org.apache.hadoop.yarn.webapp.GuiceServletConfig;
 import org.apache.hadoop.yarn.webapp.JerseyTestBase;
 import org.apache.hadoop.yarn.webapp.WebServicesTestUtils;
 import org.codehaus.jettison.json.JSONException;
@@ -107,7 +109,6 @@ import org.xml.sax.SAXException;
 import com.google.inject.Guice;
 import com.google.inject.Injector;
 import com.google.inject.Singleton;
-import com.google.inject.servlet.GuiceServletContextListener;
 import com.google.inject.servlet.ServletModule;
 import com.sun.jersey.api.client.Client;
 import com.sun.jersey.api.client.ClientResponse;
@@ -126,7 +127,6 @@ public class TestRMWebServicesAppsModification extends JerseyTestBase {
 
   private static final int CONTAINER_MB = 1024;
 
-  private static Injector injector;
   private String webserviceUserName = "testuser";
 
   private boolean setAuthFilter = false;
@@ -136,14 +136,6 @@ public class TestRMWebServicesAppsModification extends JerseyTestBase {
   private static final String FS_ALLOC_FILE = new File(TEST_DIR,
       "test-fs-queues.xml").getAbsolutePath();
 
-  public static class GuiceServletConfig extends GuiceServletContextListener {
-
-    @Override
-    protected Injector getInjector() {
-      return injector;
-    }
-  }
-
   /*
    * Helper class to allow testing of RM web services which require
    * authorization Add this class as a filter in the Guice injector for the
@@ -298,19 +290,19 @@ public class TestRMWebServicesAppsModification extends JerseyTestBase {
     case 0:
     default:
       // No Auth Capacity Scheduler
-      injector = getNoAuthInjectorCap();
+      GuiceServletConfig.setInjector(getNoAuthInjectorCap());
       break;
     case 1:
       // Simple Auth Capacity Scheduler
-      injector = getSimpleAuthInjectorCap();
+      GuiceServletConfig.setInjector(getSimpleAuthInjectorCap());
       break;
     case 2:
       // No Auth Fair Scheduler
-      injector = getNoAuthInjectorFair();
+      GuiceServletConfig.setInjector(getNoAuthInjectorFair());
       break;
     case 3:
       // Simple Auth Fair Scheduler
-      injector = getSimpleAuthInjectorFair();
+      GuiceServletConfig.setInjector(getSimpleAuthInjectorFair());
       break;
     }
   }
@@ -349,7 +341,7 @@ public class TestRMWebServicesAppsModification extends JerseyTestBase {
           this
             .constructWebResource("apps", app.getApplicationId().toString(),
               "state").accept(mediaType).get(ClientResponse.class);
-      assertEquals(Status.OK, response.getClientResponseStatus());
+      assertResponseStatusCode(Status.OK, response.getStatusInfo());
       if (mediaType.equals(MediaType.APPLICATION_JSON)) {
         verifyAppStateJson(response, RMAppState.ACCEPTED);
       } else if (mediaType.equals(MediaType.APPLICATION_XML)) {
@@ -388,10 +380,11 @@ public class TestRMWebServicesAppsModification extends JerseyTestBase {
               .put(ClientResponse.class);
 
         if (!isAuthenticationEnabled()) {
-          assertEquals(Status.UNAUTHORIZED, response.getClientResponseStatus());
+          assertResponseStatusCode(Status.UNAUTHORIZED,
+              response.getStatusInfo());
           continue;
         }
-        assertEquals(Status.ACCEPTED, response.getClientResponseStatus());
+        assertResponseStatusCode(Status.ACCEPTED, response.getStatusInfo());
         if (mediaType.equals(MediaType.APPLICATION_JSON)) {
           verifyAppStateJson(response, RMAppState.FINAL_SAVING,
             RMAppState.KILLED, RMAppState.KILLING, RMAppState.ACCEPTED);
@@ -408,7 +401,7 @@ public class TestRMWebServicesAppsModification extends JerseyTestBase {
           tmp = tmp.queryParam("user.name", webserviceUserName);
         }
         response = tmp.get(ClientResponse.class);
-        assertEquals(Status.OK, response.getClientResponseStatus());
+        assertResponseStatusCode(Status.OK, response.getStatusInfo());
         assertTrue(locationHeaderValue.endsWith("/ws/v1/cluster/apps/"
             + app.getApplicationId().toString() + "/state"));
 
@@ -419,9 +412,13 @@ public class TestRMWebServicesAppsModification extends JerseyTestBase {
                 .constructWebResource("apps",
                   app.getApplicationId().toString(), "state").accept(mediaType)
                 .entity(entity, contentType).put(ClientResponse.class);
-          assertTrue((response.getClientResponseStatus() == Status.ACCEPTED)
-              || (response.getClientResponseStatus() == Status.OK));
-          if (response.getClientResponseStatus() == Status.OK) {
+          assertTrue(
+              (response.getStatusInfo().getStatusCode()
+                  == Status.ACCEPTED.getStatusCode())
+              || (response.getStatusInfo().getStatusCode()
+                  == Status.OK.getStatusCode()));
+          if (response.getStatusInfo().getStatusCode()
+              == Status.OK.getStatusCode()) {
             assertEquals(RMAppState.KILLED, app.getState());
             if (mediaType.equals(MediaType.APPLICATION_JSON)) {
               verifyAppStateJson(response, RMAppState.KILLED);
@@ -470,11 +467,12 @@ public class TestRMWebServicesAppsModification extends JerseyTestBase {
                 .put(ClientResponse.class);
 
           if (!isAuthenticationEnabled()) {
-            assertEquals(Status.UNAUTHORIZED,
-              response.getClientResponseStatus());
+            assertResponseStatusCode(Status.UNAUTHORIZED,
+                response.getStatusInfo());
             continue;
           }
-          assertEquals(Status.BAD_REQUEST, response.getClientResponseStatus());
+          assertResponseStatusCode(Status.BAD_REQUEST,
+              response.getStatusInfo());
         }
       }
     }
@@ -588,13 +586,16 @@ public class TestRMWebServicesAppsModification extends JerseyTestBase {
             .accept(MediaType.APPLICATION_XML)
             .entity(info, MediaType.APPLICATION_XML).put(ClientResponse.class);
       if (!isAuthenticationEnabled()) {
-        assertEquals(Status.UNAUTHORIZED, response.getClientResponseStatus());
+        assertResponseStatusCode(Status.UNAUTHORIZED,
+            response.getStatusInfo());
         continue;
       }
       if (i == 0) {
-        assertEquals(Status.NOT_FOUND, response.getClientResponseStatus());
+        assertResponseStatusCode(Status.NOT_FOUND,
+            response.getStatusInfo());
       } else {
-        assertEquals(Status.BAD_REQUEST, response.getClientResponseStatus());
+        assertResponseStatusCode(Status.BAD_REQUEST,
+            response.getStatusInfo());
       }
     }
     rm.stop();
@@ -641,9 +642,11 @@ public class TestRMWebServicesAppsModification extends JerseyTestBase {
   public void validateResponseStatus(ClientResponse response,
       Status expectedUnauthorizedMode, Status expectedAuthorizedMode) {
     if (!isAuthenticationEnabled()) {
-      assertEquals(expectedUnauthorizedMode, response.getClientResponseStatus());
+      assertResponseStatusCode(expectedUnauthorizedMode,
+          response.getStatusInfo());
     } else {
-      assertEquals(expectedAuthorizedMode, response.getClientResponseStatus());
+      assertResponseStatusCode(expectedAuthorizedMode,
+          response.getStatusInfo());
     }
   }
 
@@ -845,10 +848,10 @@ public class TestRMWebServicesAppsModification extends JerseyTestBase {
           .entity(appInfo, contentMedia).post(ClientResponse.class);
 
     if (!this.isAuthenticationEnabled()) {
-      assertEquals(Status.UNAUTHORIZED, response.getClientResponseStatus());
+      assertResponseStatusCode(Status.UNAUTHORIZED, response.getStatusInfo());
       return;
     }
-    assertEquals(Status.ACCEPTED, response.getClientResponseStatus());
+    assertResponseStatusCode(Status.ACCEPTED, response.getStatusInfo());
     assertTrue(!response.getHeaders().getFirst(HttpHeaders.LOCATION).isEmpty());
     String locURL = response.getHeaders().getFirst(HttpHeaders.LOCATION);
     assertTrue(locURL.contains("/apps/application"));
@@ -857,7 +860,7 @@ public class TestRMWebServicesAppsModification extends JerseyTestBase {
     WebResource res = resource().uri(new URI(locURL));
     res = res.queryParam("user.name", webserviceUserName);
     response = res.get(ClientResponse.class);
-    assertEquals(Status.OK, response.getClientResponseStatus());
+    assertResponseStatusCode(Status.OK, response.getStatusInfo());
 
     RMApp app =
         rm.getRMContext().getRMApps()
@@ -916,7 +919,7 @@ public class TestRMWebServicesAppsModification extends JerseyTestBase {
     response =
         this.constructWebResource("apps", appId).accept(acceptMedia)
           .get(ClientResponse.class);
-    assertEquals(Status.OK, response.getClientResponseStatus());
+    assertResponseStatusCode(Status.OK, response.getStatusInfo());
   }
 
   public void testAppSubmitErrors(String acceptMedia, String contentMedia)
@@ -1004,7 +1007,7 @@ public class TestRMWebServicesAppsModification extends JerseyTestBase {
     ClientResponse response =
         this.constructWebResource(urlPath).accept(MediaType.APPLICATION_XML)
           .entity(body, MediaType.APPLICATION_XML).post(ClientResponse.class);
-    assertEquals(Status.BAD_REQUEST, response.getClientResponseStatus());
+    assertResponseStatusCode(Status.BAD_REQUEST, response.getStatusInfo());
     body = "{\"a\" : \"b\"}";
     response =
         this.constructWebResource(urlPath).accept(MediaType.APPLICATION_XML)
@@ -1029,7 +1032,7 @@ public class TestRMWebServicesAppsModification extends JerseyTestBase {
           this
             .constructWebResource("apps", app.getApplicationId().toString(),
               "queue").accept(contentType).get(ClientResponse.class);
-      assertEquals(Status.OK, response.getClientResponseStatus());
+      assertResponseStatusCode(Status.OK, response.getStatusInfo());
       String expectedQueue = "default";
       if(!isCapacityScheduler) {
         expectedQueue = "root." + webserviceUserName;
@@ -1094,10 +1097,11 @@ public class TestRMWebServicesAppsModification extends JerseyTestBase {
             .put(ClientResponse.class);
 
         if (!isAuthenticationEnabled()) {
-          assertEquals(Status.UNAUTHORIZED, response.getClientResponseStatus());
+          assertResponseStatusCode(Status.UNAUTHORIZED,
+              response.getStatusInfo());
           continue;
         }
-        assertEquals(Status.OK, response.getClientResponseStatus());
+        assertResponseStatusCode(Status.OK, response.getStatusInfo());
         if (mediaType.equals(MediaType.APPLICATION_JSON)) {
           verifyAppPriorityJson(response, modifiedPriority);
         } else {
@@ -1108,7 +1112,7 @@ public class TestRMWebServicesAppsModification extends JerseyTestBase {
             .constructWebResource("apps", app.getApplicationId().toString(),
                 "priority")
             .accept(mediaType).get(ClientResponse.class);
-        assertEquals(Status.OK, response.getClientResponseStatus());
+        assertResponseStatusCode(Status.OK, response.getStatusInfo());
         if (mediaType.equals(MediaType.APPLICATION_JSON)) {
           verifyAppPriorityJson(response, modifiedPriority);
         } else {
@@ -1123,7 +1127,7 @@ public class TestRMWebServicesAppsModification extends JerseyTestBase {
                 "priority")
             .entity(entity, contentType).accept(mediaType)
             .put(ClientResponse.class);
-        assertEquals(Status.FORBIDDEN, response.getClientResponseStatus());
+        assertResponseStatusCode(Status.FORBIDDEN, response.getStatusInfo());
       }
     }
     rm.stop();
@@ -1173,10 +1177,11 @@ public class TestRMWebServicesAppsModification extends JerseyTestBase {
               .put(ClientResponse.class);
 
         if (!isAuthenticationEnabled()) {
-          assertEquals(Status.UNAUTHORIZED, response.getClientResponseStatus());
+          assertResponseStatusCode(Status.UNAUTHORIZED,
+              response.getStatusInfo());
           continue;
         }
-        assertEquals(Status.OK, response.getClientResponseStatus());
+        assertResponseStatusCode(Status.OK, response.getStatusInfo());
         String expectedQueue = "test";
         if(!isCapacityScheduler) {
           expectedQueue = "root.test";
@@ -1196,7 +1201,7 @@ public class TestRMWebServicesAppsModification extends JerseyTestBase {
               .constructWebResource("apps", app.getApplicationId().toString(),
                 "queue").entity(entity, contentType).accept(mediaType)
               .put(ClientResponse.class);
-        assertEquals(Status.FORBIDDEN, response.getClientResponseStatus());
+        assertResponseStatusCode(Status.FORBIDDEN, response.getStatusInfo());
         if(isCapacityScheduler) {
           Assert.assertEquals("default", app.getQueue());
         }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5d58858b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesCapacitySched.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/TestRMWebServicesCapacitySched.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesCapacitySched.java
index 9379367..649d719 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesCapacitySched.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesCapacitySched.java
@@ -38,6 +38,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.Capacity
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration;
 import org.apache.hadoop.yarn.util.resource.Resources;
 import org.apache.hadoop.yarn.webapp.GenericExceptionHandler;
+import org.apache.hadoop.yarn.webapp.GuiceServletConfig;
 import org.apache.hadoop.yarn.webapp.JerseyTestBase;
 import org.apache.hadoop.yarn.webapp.WebServicesTestUtils;
 import org.codehaus.jettison.json.JSONArray;
@@ -53,8 +54,6 @@ import org.w3c.dom.NodeList;
 import org.xml.sax.InputSource;
 
 import com.google.inject.Guice;
-import com.google.inject.Injector;
-import com.google.inject.servlet.GuiceServletContextListener;
 import com.google.inject.servlet.ServletModule;
 import com.sun.jersey.api.client.ClientResponse;
 import com.sun.jersey.api.client.WebResource;
@@ -64,8 +63,8 @@ import com.sun.jersey.test.framework.WebAppDescriptor;
 public class TestRMWebServicesCapacitySched extends JerseyTestBase {
 
   private static MockRM rm;
-  private CapacitySchedulerConfiguration csConf;
-  private YarnConfiguration conf;
+  private static CapacitySchedulerConfiguration csConf;
+  private static YarnConfiguration conf;
 
   private class QueueInfo {
     float capacity;
@@ -89,7 +88,7 @@ public class TestRMWebServicesCapacitySched extends JerseyTestBase {
     float userLimitFactor;
   }
 
-  private Injector injector = Guice.createInjector(new ServletModule() {
+  private static class WebServletModule extends ServletModule {
     @Override
     protected void configureServlets() {
       bind(JAXBContextResolver.class);
@@ -104,62 +103,62 @@ public class TestRMWebServicesCapacitySched extends JerseyTestBase {
       bind(ResourceManager.class).toInstance(rm);
       serve("/*").with(GuiceContainer.class);
     }
-  });
-
-  public class GuiceServletConfig extends GuiceServletContextListener {
+  }
 
-    @Override
-    protected Injector getInjector() {
-      return injector;
-    }
+  static {
+    GuiceServletConfig.setInjector(
+        Guice.createInjector(new WebServletModule()));
   }
 
   private static void setupQueueConfiguration(
-      CapacitySchedulerConfiguration conf) {
+      CapacitySchedulerConfiguration config) {
 
     // Define top-level queues
-    conf.setQueues(CapacitySchedulerConfiguration.ROOT, new String[] { "a", "b" });
+    config.setQueues(CapacitySchedulerConfiguration.ROOT,
+        new String[] {"a", "b"});
 
     final String A = CapacitySchedulerConfiguration.ROOT + ".a";
-    conf.setCapacity(A, 10.5f);
-    conf.setMaximumCapacity(A, 50);
+    config.setCapacity(A, 10.5f);
+    config.setMaximumCapacity(A, 50);
 
     final String B = CapacitySchedulerConfiguration.ROOT + ".b";
-    conf.setCapacity(B, 89.5f);
+    config.setCapacity(B, 89.5f);
 
     // Define 2nd-level queues
     final String A1 = A + ".a1";
     final String A2 = A + ".a2";
-    conf.setQueues(A, new String[] { "a1", "a2" });
-    conf.setCapacity(A1, 30);
-    conf.setMaximumCapacity(A1, 50);
+    config.setQueues(A, new String[] {"a1", "a2"});
+    config.setCapacity(A1, 30);
+    config.setMaximumCapacity(A1, 50);
 
-    conf.setUserLimitFactor(A1, 100.0f);
-    conf.setCapacity(A2, 70);
-    conf.setUserLimitFactor(A2, 100.0f);
+    config.setUserLimitFactor(A1, 100.0f);
+    config.setCapacity(A2, 70);
+    config.setUserLimitFactor(A2, 100.0f);
 
     final String B1 = B + ".b1";
     final String B2 = B + ".b2";
     final String B3 = B + ".b3";
-    conf.setQueues(B, new String[] { "b1", "b2", "b3" });
-    conf.setCapacity(B1, 60);
-    conf.setUserLimitFactor(B1, 100.0f);
-    conf.setCapacity(B2, 39.5f);
-    conf.setUserLimitFactor(B2, 100.0f);
-    conf.setCapacity(B3, 0.5f);
-    conf.setUserLimitFactor(B3, 100.0f);
+    config.setQueues(B, new String[] {"b1", "b2", "b3"});
+    config.setCapacity(B1, 60);
+    config.setUserLimitFactor(B1, 100.0f);
+    config.setCapacity(B2, 39.5f);
+    config.setUserLimitFactor(B2, 100.0f);
+    config.setCapacity(B3, 0.5f);
+    config.setUserLimitFactor(B3, 100.0f);
     
-    conf.setQueues(A1, new String[] {"a1a", "a1b"});
+    config.setQueues(A1, new String[] {"a1a", "a1b"});
     final String A1A = A1 + ".a1a";
-    conf.setCapacity(A1A, 85);
+    config.setCapacity(A1A, 85);
     final String A1B = A1 + ".a1b";
-    conf.setCapacity(A1B, 15);
+    config.setCapacity(A1B, 15);
   }
 
   @Before
   @Override
   public void setUp() throws Exception {
     super.setUp();
+    GuiceServletConfig.setInjector(
+        Guice.createInjector(new WebServletModule()));
   }
 
   public TestRMWebServicesCapacitySched() {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5d58858b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesDelegationTokens.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/TestRMWebServicesDelegationTokens.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesDelegationTokens.java
index a6d7744..fe01d28 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesDelegationTokens.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesDelegationTokens.java
@@ -18,6 +18,8 @@
 
 package org.apache.hadoop.yarn.server.resourcemanager.webapp;
 
+import static org.apache.hadoop.yarn.webapp.WebServicesTestUtils.assertResponseStatusCode;
+
 import java.io.File;
 import java.io.IOException;
 import java.io.StringReader;
@@ -54,6 +56,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fifo.FifoScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.DelegationToken;
 import org.apache.hadoop.yarn.webapp.GenericExceptionHandler;
+import org.apache.hadoop.yarn.webapp.GuiceServletConfig;
 import org.apache.hadoop.yarn.webapp.JerseyTestBase;
 import org.apache.hadoop.yarn.webapp.WebServicesTestUtils;
 import org.codehaus.jettison.json.JSONException;
@@ -79,7 +82,6 @@ import static org.junit.Assert.assertFalse;
 import com.google.inject.Guice;
 import com.google.inject.Injector;
 import com.google.inject.Singleton;
-import com.google.inject.servlet.GuiceServletContextListener;
 import com.google.inject.servlet.ServletModule;
 import com.sun.jersey.api.client.ClientResponse;
 import com.sun.jersey.api.client.ClientResponse.Status;
@@ -97,7 +99,6 @@ public class TestRMWebServicesDelegationTokens extends JerseyTestBase {
     .getServerPrincipal();
   private static MiniKdc testMiniKDC;
   private static MockRM rm;
-  private Injector injector;
   private boolean isKerberosAuth = false;
 
   // Make sure the test uses the published header string
@@ -200,14 +201,6 @@ public class TestRMWebServicesDelegationTokens extends JerseyTestBase {
     });
   }
 
-  public class GuiceServletConfig extends GuiceServletContextListener {
-
-    @Override
-    protected Injector getInjector() {
-      return injector;
-    }
-  }
-
   @Parameters
   public static Collection<Object[]> guiceConfigs() {
     return Arrays.asList(new Object[][] { { 0 }, { 1 } });
@@ -222,10 +215,10 @@ public class TestRMWebServicesDelegationTokens extends JerseyTestBase {
     switch (run) {
     case 0:
     default:
-      injector = getKerberosAuthInjector();
+      GuiceServletConfig.setInjector(getKerberosAuthInjector());
       break;
     case 1:
-      injector = getSimpleAuthInjector();
+      GuiceServletConfig.setInjector(getSimpleAuthInjector());
       break;
     }
   }
@@ -305,7 +298,7 @@ public class TestRMWebServicesDelegationTokens extends JerseyTestBase {
           .path("delegation-token").queryParam("user.name", "testuser")
           .accept(contentType).entity(body, mediaType)
           .post(ClientResponse.class);
-    assertEquals(Status.FORBIDDEN, response.getClientResponseStatus());
+    assertResponseStatusCode(Status.FORBIDDEN, response.getStatusInfo());
   }
 
   private void verifyKerberosAuthCreate(String mType, String cType,
@@ -321,7 +314,7 @@ public class TestRMWebServicesDelegationTokens extends JerseyTestBase {
             resource().path("ws").path("v1").path("cluster")
               .path("delegation-token").accept(contentType)
               .entity(body, mediaType).post(ClientResponse.class);
-        assertEquals(Status.OK, response.getClientResponseStatus());
+        assertResponseStatusCode(Status.OK, response.getStatusInfo());
         DelegationToken tok = getDelegationTokenFromResponse(response);
         assertFalse(tok.getToken().isEmpty());
         Token<RMDelegationTokenIdentifier> token =
@@ -334,7 +327,7 @@ public class TestRMWebServicesDelegationTokens extends JerseyTestBase {
             resource().path("ws").path("v1").path("cluster")
               .path("delegation-token").accept(contentType)
               .entity(dtoken, mediaType).post(ClientResponse.class);
-        assertEquals(Status.OK, response.getClientResponseStatus());
+        assertResponseStatusCode(Status.OK, response.getStatusInfo());
         tok = getDelegationTokenFromResponse(response);
         assertFalse(tok.getToken().isEmpty());
         token = new Token<RMDelegationTokenIdentifier>();
@@ -375,7 +368,7 @@ public class TestRMWebServicesDelegationTokens extends JerseyTestBase {
                     resource().path("ws").path("v1").path("cluster")
                       .path("delegation-token").accept(contentType)
                       .entity(dummyToken, mediaType).post(ClientResponse.class);
-                assertEquals(Status.OK, response.getClientResponseStatus());
+                assertResponseStatusCode(Status.OK, response.getStatusInfo());
                 DelegationToken tok = getDelegationTokenFromResponse(response);
                 assertFalse(tok.getToken().isEmpty());
                 String body = generateRenewTokenBody(mediaType, tok.getToken());
@@ -385,8 +378,8 @@ public class TestRMWebServicesDelegationTokens extends JerseyTestBase {
                       .header(yarnTokenHeader, tok.getToken())
                       .accept(contentType).entity(body, mediaType)
                       .post(ClientResponse.class);
-                assertEquals(Status.FORBIDDEN,
-                  response.getClientResponseStatus());
+                assertResponseStatusCode(Status.FORBIDDEN,
+                    response.getStatusInfo());
                 return tok;
               }
             });
@@ -406,7 +399,7 @@ public class TestRMWebServicesDelegationTokens extends JerseyTestBase {
                   .header(yarnTokenHeader, responseToken.getToken())
                   .accept(contentType).entity(body, mediaType)
                   .post(ClientResponse.class);
-            assertEquals(Status.OK, response.getClientResponseStatus());
+            assertResponseStatusCode(Status.OK, response.getStatusInfo());
             DelegationToken tok = getDelegationTokenFromResponse(response);
             String message =
                 "Expiration time not as expected: old = " + oldExpirationTime
@@ -421,7 +414,7 @@ public class TestRMWebServicesDelegationTokens extends JerseyTestBase {
                   .header(yarnTokenHeader, responseToken.getToken())
                   .accept(contentType).entity(body, mediaType)
                   .post(ClientResponse.class);
-            assertEquals(Status.OK, response.getClientResponseStatus());
+            assertResponseStatusCode(Status.OK, response.getStatusInfo());
             tok = getDelegationTokenFromResponse(response);
             message =
                 "Expiration time not as expected: old = " + oldExpirationTime
@@ -443,7 +436,8 @@ public class TestRMWebServicesDelegationTokens extends JerseyTestBase {
                   .header(yarnTokenHeader, responseToken.getToken())
                   .accept(contentType).entity(body, mediaType)
                   .post(ClientResponse.class);
-            assertEquals(Status.FORBIDDEN, response.getClientResponseStatus());
+            assertResponseStatusCode(Status.FORBIDDEN,
+                response.getStatusInfo());
             return null;
           }
         });
@@ -469,7 +463,8 @@ public class TestRMWebServicesDelegationTokens extends JerseyTestBase {
                   .path("delegation-token").path("expiration")
                   .accept(contentType).entity(body, mediaType)
                   .post(ClientResponse.class);
-            assertEquals(Status.BAD_REQUEST, response.getClientResponseStatus());
+            assertResponseStatusCode(Status.BAD_REQUEST,
+                response.getStatusInfo());
             return null;
           }
         });
@@ -498,7 +493,7 @@ public class TestRMWebServicesDelegationTokens extends JerseyTestBase {
           .path("delegation-token").queryParam("user.name", "testuser")
           .accept(contentType).entity(body, mediaType)
           .post(ClientResponse.class);
-    assertEquals(Status.FORBIDDEN, response.getClientResponseStatus());
+    assertResponseStatusCode(Status.FORBIDDEN, response.getStatusInfo());
   }
 
   // Test to verify cancel functionality - create a token and then try to cancel
@@ -528,14 +523,14 @@ public class TestRMWebServicesDelegationTokens extends JerseyTestBase {
                 resource().path("ws").path("v1").path("cluster")
                   .path("delegation-token").accept(contentType)
                   .entity(dtoken, mediaType).post(ClientResponse.class);
-            assertEquals(Status.OK, response.getClientResponseStatus());
+            assertResponseStatusCode(Status.OK, response.getStatusInfo());
             DelegationToken tok = getDelegationTokenFromResponse(response);
             response =
                 resource().path("ws").path("v1").path("cluster")
                   .path("delegation-token")
                   .header(yarnTokenHeader, tok.getToken()).accept(contentType)
                   .delete(ClientResponse.class);
-            assertEquals(Status.OK, response.getClientResponseStatus());
+            assertResponseStatusCode(Status.OK, response.getStatusInfo());
             assertTokenCancelled(tok.getToken());
             return null;
           }
@@ -550,7 +545,7 @@ public class TestRMWebServicesDelegationTokens extends JerseyTestBase {
                     resource().path("ws").path("v1").path("cluster")
                       .path("delegation-token").accept(contentType)
                       .entity(dtoken, mediaType).post(ClientResponse.class);
-                assertEquals(Status.OK, response.getClientResponseStatus());
+                assertResponseStatusCode(Status.OK, response.getStatusInfo());
                 DelegationToken tok = getDelegationTokenFromResponse(response);
                 return tok;
               }
@@ -564,7 +559,7 @@ public class TestRMWebServicesDelegationTokens extends JerseyTestBase {
                   .path("delegation-token")
                   .header(yarnTokenHeader, tmpToken.getToken())
                   .accept(contentType).delete(ClientResponse.class);
-            assertEquals(Status.OK, response.getClientResponseStatus());
+            assertResponseStatusCode(Status.OK, response.getStatusInfo());
             assertTokenCancelled(tmpToken.getToken());
             return null;
           }
@@ -579,7 +574,7 @@ public class TestRMWebServicesDelegationTokens extends JerseyTestBase {
                     resource().path("ws").path("v1").path("cluster")
                       .path("delegation-token").accept(contentType)
                       .entity(dtoken, mediaType).post(ClientResponse.class);
-                assertEquals(Status.OK, response.getClientResponseStatus());
+                assertResponseStatusCode(Status.OK, response.getStatusInfo());
                 DelegationToken tok = getDelegationTokenFromResponse(response);
                 return tok;
               }
@@ -593,7 +588,8 @@ public class TestRMWebServicesDelegationTokens extends JerseyTestBase {
                   .path("delegation-token")
                   .header(yarnTokenHeader, tmpToken2.getToken())
                   .accept(contentType).delete(ClientResponse.class);
-            assertEquals(Status.FORBIDDEN, response.getClientResponseStatus());
+            assertResponseStatusCode(Status.FORBIDDEN,
+                response.getStatusInfo());
             assertValidRMToken(tmpToken2.getToken());
             return null;
           }
@@ -625,7 +621,7 @@ public class TestRMWebServicesDelegationTokens extends JerseyTestBase {
               .path("delegation-token")
               .header(yarnTokenHeader, "random-string").accept(contentType)
               .delete(ClientResponse.class);
-        assertEquals(Status.BAD_REQUEST, response.getClientResponseStatus());
+        assertResponseStatusCode(Status.BAD_REQUEST, response.getStatusInfo());
         return null;
       }
     });
@@ -638,7 +634,7 @@ public class TestRMWebServicesDelegationTokens extends JerseyTestBase {
             resource().path("ws").path("v1").path("cluster")
               .path("delegation-token").accept(contentType)
               .delete(ClientResponse.class);
-        assertEquals(Status.BAD_REQUEST, response.getClientResponseStatus());
+        assertResponseStatusCode(Status.BAD_REQUEST, response.getStatusInfo());
 
         return null;
       }
@@ -653,7 +649,7 @@ public class TestRMWebServicesDelegationTokens extends JerseyTestBase {
                 resource().path("ws").path("v1").path("cluster")
                   .path("delegation-token").accept(contentType)
                   .entity(dtoken, mediaType).post(ClientResponse.class);
-            assertEquals(Status.OK, response.getClientResponseStatus());
+            assertResponseStatusCode(Status.OK, response.getStatusInfo());
             DelegationToken tok = getDelegationTokenFromResponse(response);
             return tok;
           }
@@ -667,13 +663,13 @@ public class TestRMWebServicesDelegationTokens extends JerseyTestBase {
               .path("delegation-token")
               .header(yarnTokenHeader, tmpToken.getToken()).accept(contentType)
               .delete(ClientResponse.class);
-        assertEquals(Status.OK, response.getClientResponseStatus());
+        assertResponseStatusCode(Status.OK, response.getStatusInfo());
         response =
             resource().path("ws").path("v1").path("cluster")
               .path("delegation-token")
               .header(yarnTokenHeader, tmpToken.getToken()).accept(contentType)
               .delete(ClientResponse.class);
-        assertEquals(Status.BAD_REQUEST, response.getClientResponseStatus());
+        assertResponseStatusCode(Status.BAD_REQUEST, response.getStatusInfo());
         return null;
       }
     });
@@ -686,7 +682,7 @@ public class TestRMWebServicesDelegationTokens extends JerseyTestBase {
           .path("delegation-token").queryParam("user.name", "testuser")
           .header(RMWebServices.DELEGATION_TOKEN_HEADER, "random")
           .delete(ClientResponse.class);
-    assertEquals(Status.FORBIDDEN, response.getClientResponseStatus());
+    assertResponseStatusCode(Status.FORBIDDEN, response.getStatusInfo());
   }
 
   private DelegationToken


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


[08/17] hadoop git commit: HADOOP-9613. [JDK8] Update jersey version to latest 1.x release.

Posted by ae...@apache.org.
HADOOP-9613. [JDK8] Update jersey version to latest 1.x release.


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

Branch: refs/heads/HDFS-1312
Commit: 5d58858bb6dfc07272ef099d60ca7cfb3b04423c
Parents: 8c1f81d
Author: Tsuyoshi Ozawa <oz...@apache.org>
Authored: Tue Feb 16 01:02:09 2016 +0900
Committer: Tsuyoshi Ozawa <oz...@apache.org>
Committed: Tue Jun 21 08:05:32 2016 +0900

----------------------------------------------------------------------
 hadoop-common-project/hadoop-common/pom.xml     |   5 +
 .../v2/app/webapp/TestAMWebServices.java        |  29 ++--
 .../v2/app/webapp/TestAMWebServicesAttempt.java |  20 ++-
 .../app/webapp/TestAMWebServicesAttempts.java   |  24 ++--
 .../v2/app/webapp/TestAMWebServicesJobConf.java |  27 ++--
 .../v2/app/webapp/TestAMWebServicesJobs.java    |  31 ++---
 .../v2/app/webapp/TestAMWebServicesTasks.java   |  32 ++---
 .../v2/hs/webapp/TestHsWebServices.java         |  28 ++--
 .../v2/hs/webapp/TestHsWebServicesAttempts.java |  33 ++---
 .../v2/hs/webapp/TestHsWebServicesJobConf.java  |  20 ++-
 .../v2/hs/webapp/TestHsWebServicesJobs.java     |  35 +++--
 .../hs/webapp/TestHsWebServicesJobsQuery.java   |  60 ++++----
 .../v2/hs/webapp/TestHsWebServicesTasks.java    |  32 ++---
 hadoop-project/pom.xml                          |  19 ++-
 .../apache/hadoop/yarn/client/cli/LogsCLI.java  |   3 +-
 .../hadoop-yarn/hadoop-yarn-common/pom.xml      |   6 +
 .../yarn/client/api/impl/TimelineWriter.java    |   3 +-
 .../client/api/impl/TestTimelineClient.java     |   4 +-
 .../api/impl/TestTimelineClientForATS1_5.java   |   4 +-
 .../hadoop/yarn/webapp/GuiceServletConfig.java  |  43 ++++++
 .../yarn/webapp/WebServicesTestUtils.java       |  11 ++
 .../webapp/TestAHSWebServices.java              |  64 ++++-----
 .../webapp/TestTimelineWebServices.java         |  56 ++++----
 .../nodemanager/webapp/TestNMWebServices.java   |  29 ++--
 .../webapp/TestNMWebServicesApps.java           |  42 +++---
 .../webapp/TestNMWebServicesContainers.java     |  29 ++--
 .../webapp/TestRMWebServices.java               |  29 ++--
 .../webapp/TestRMWebServicesApps.java           |  51 +++----
 .../TestRMWebServicesAppsModification.java      |  89 ++++++------
 .../webapp/TestRMWebServicesCapacitySched.java  |  65 +++++----
 .../TestRMWebServicesDelegationTokens.java      |  66 +++++----
 .../webapp/TestRMWebServicesFairScheduler.java  |  29 ++--
 .../TestRMWebServicesForCSWithPartitions.java   |  92 ++++++------
 .../webapp/TestRMWebServicesNodeLabels.java     |  30 ++--
 .../webapp/TestRMWebServicesNodes.java          |  35 ++---
 .../webapp/TestRMWebServicesReservation.java    | 139 ++++++++++---------
 .../yarn/webapp/TestRMWithCSRFFilter.java       |  22 +--
 .../hadoop/yarn/webapp/TestRMWithXFSFilter.java |  19 +--
 38 files changed, 692 insertions(+), 663 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/5d58858b/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 51ff7de..c21b81e 100644
--- a/hadoop-common-project/hadoop-common/pom.xml
+++ b/hadoop-common-project/hadoop-common/pom.xml
@@ -116,6 +116,11 @@
       <scope>compile</scope>
     </dependency>
     <dependency>
+      <groupId>com.sun.jersey</groupId>
+      <artifactId>jersey-servlet</artifactId>
+      <scope>compile</scope>
+    </dependency>
+    <dependency>
       <!-- Used, even though 'mvn dependency:analyze' doesn't find it -->
       <groupId>com.sun.jersey</groupId>
       <artifactId>jersey-json</artifactId>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5d58858b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/webapp/TestAMWebServices.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/webapp/TestAMWebServices.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/webapp/TestAMWebServices.java
index 21a2611..1fb6a08 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/webapp/TestAMWebServices.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/webapp/TestAMWebServices.java
@@ -18,6 +18,7 @@
 
 package org.apache.hadoop.mapreduce.v2.app.webapp;
 
+import static org.apache.hadoop.yarn.webapp.WebServicesTestUtils.assertResponseStatusCode;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
@@ -33,6 +34,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.mapreduce.v2.app.AppContext;
 import org.apache.hadoop.mapreduce.v2.app.MockAppContext;
 import org.apache.hadoop.yarn.webapp.GenericExceptionHandler;
+import org.apache.hadoop.yarn.webapp.GuiceServletConfig;
 import org.apache.hadoop.yarn.webapp.WebServicesTestUtils;
 import org.codehaus.jettison.json.JSONArray;
 import org.codehaus.jettison.json.JSONException;
@@ -46,8 +48,6 @@ import org.xml.sax.InputSource;
 
 import com.google.common.collect.Sets;
 import com.google.inject.Guice;
-import com.google.inject.Injector;
-import com.google.inject.servlet.GuiceServletContextListener;
 import com.google.inject.servlet.ServletModule;
 import com.sun.jersey.api.client.ClientResponse;
 import com.sun.jersey.api.client.ClientResponse.Status;
@@ -69,13 +69,13 @@ public class TestAMWebServices extends JerseyTest {
   private static Configuration conf = new Configuration();
   private static MockAppContext appContext;
 
-  private Injector injector = Guice.createInjector(new ServletModule() {
+  private static class WebServletModule extends ServletModule {
     @Override
     protected void configureServlets() {
 
       appContext = new MockAppContext(0, 1, 1, 1);
       appContext.setBlacklistedNodes(Sets.newHashSet("badnode1", "badnode2"));
-      
+
       bind(JAXBContextResolver.class);
       bind(AMWebServices.class);
       bind(GenericExceptionHandler.class);
@@ -84,20 +84,19 @@ public class TestAMWebServices extends JerseyTest {
 
       serve("/*").with(GuiceContainer.class);
     }
-  });
-
-  public class GuiceServletConfig extends GuiceServletContextListener {
+  }
 
-    @Override
-    protected Injector getInjector() {
-      return injector;
-    }
+  static {
+    GuiceServletConfig.setInjector(
+        Guice.createInjector(new WebServletModule()));
   }
 
   @Before
   @Override
   public void setUp() throws Exception {
     super.setUp();
+    GuiceServletConfig.setInjector(
+        Guice.createInjector(new WebServletModule()));
   }
 
   public TestAMWebServices() {
@@ -207,7 +206,7 @@ public class TestAMWebServices extends JerseyTest {
       fail("should have thrown exception on invalid uri");
     } catch (UniformInterfaceException ue) {
       ClientResponse response = ue.getResponse();
-      assertEquals(Status.NOT_FOUND, response.getClientResponseStatus());
+      assertResponseStatusCode(Status.NOT_FOUND, response.getStatusInfo());
       WebServicesTestUtils.checkStringMatch(
           "error string exists and shouldn't", "", responseStr);
     }
@@ -223,7 +222,7 @@ public class TestAMWebServices extends JerseyTest {
       fail("should have thrown exception on invalid uri");
     } catch (UniformInterfaceException ue) {
       ClientResponse response = ue.getResponse();
-      assertEquals(Status.NOT_FOUND, response.getClientResponseStatus());
+      assertResponseStatusCode(Status.NOT_FOUND, response.getStatusInfo());
       WebServicesTestUtils.checkStringMatch(
           "error string exists and shouldn't", "", responseStr);
     }
@@ -239,8 +238,8 @@ public class TestAMWebServices extends JerseyTest {
       fail("should have thrown exception on invalid uri");
     } catch (UniformInterfaceException ue) {
       ClientResponse response = ue.getResponse();
-      assertEquals(Status.INTERNAL_SERVER_ERROR,
-          response.getClientResponseStatus());
+      assertResponseStatusCode(Status.INTERNAL_SERVER_ERROR,
+          response.getStatusInfo());
       WebServicesTestUtils.checkStringMatch(
           "error string exists and shouldn't", "", responseStr);
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5d58858b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/webapp/TestAMWebServicesAttempt.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/webapp/TestAMWebServicesAttempt.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/webapp/TestAMWebServicesAttempt.java
index 884e66b..68bb964 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/webapp/TestAMWebServicesAttempt.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/webapp/TestAMWebServicesAttempt.java
@@ -44,8 +44,8 @@ import org.apache.hadoop.mapreduce.v2.util.MRApps;
 import org.apache.hadoop.security.authentication.server.AuthenticationFilter;
 import org.apache.hadoop.security.authentication.server.PseudoAuthenticationHandler;
 import org.apache.hadoop.yarn.webapp.GenericExceptionHandler;
+import org.apache.hadoop.yarn.webapp.GuiceServletConfig;
 import org.apache.hadoop.yarn.webapp.WebServicesTestUtils;
-import org.codehaus.jettison.json.JSONException;
 import org.codehaus.jettison.json.JSONObject;
 import org.junit.Before;
 import org.junit.Test;
@@ -55,9 +55,7 @@ import org.w3c.dom.NodeList;
 import org.xml.sax.InputSource;
 
 import com.google.inject.Guice;
-import com.google.inject.Injector;
 import com.google.inject.Singleton;
-import com.google.inject.servlet.GuiceServletContextListener;
 import com.google.inject.servlet.ServletModule;
 import com.sun.jersey.api.client.ClientResponse;
 import com.sun.jersey.api.client.WebResource;
@@ -77,7 +75,8 @@ public class TestAMWebServicesAttempt extends JerseyTest {
   private static AppContext appContext;
   private String webserviceUserName = "testuser";
 
-  private Injector injector = Guice.createInjector(new ServletModule() {
+  private static class WebServletModule extends ServletModule {
+
     @Override
     protected void configureServlets() {
       appContext = new MockAppContext(0, 1, 2, 1);
@@ -90,7 +89,7 @@ public class TestAMWebServicesAttempt extends JerseyTest {
       serve("/*").with(GuiceContainer.class);
       filter("/*").through(TestRMCustomAuthFilter.class);
     }
-  });
+  };
 
   @Singleton
   public static class TestRMCustomAuthFilter extends AuthenticationFilter {
@@ -112,18 +111,17 @@ public class TestAMWebServicesAttempt extends JerseyTest {
     }
   }
 
-
-  public class GuiceServletConfig extends GuiceServletContextListener {
-    @Override
-    protected Injector getInjector() {
-      return injector;
-    }
+  static {
+    GuiceServletConfig.setInjector(
+        Guice.createInjector(new WebServletModule()));
   }
 
   @Before
   @Override
   public void setUp() throws Exception {
     super.setUp();
+    GuiceServletConfig.setInjector(
+        Guice.createInjector(new WebServletModule()));
   }
 
   public TestAMWebServicesAttempt() {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5d58858b/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 3c9127f..9f2e1b1 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
@@ -19,6 +19,7 @@
 
 package org.apache.hadoop.mapreduce.v2.app.webapp;
 
+import static org.apache.hadoop.yarn.webapp.WebServicesTestUtils.assertResponseStatusCode;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertTrue;
@@ -44,6 +45,7 @@ import org.apache.hadoop.mapreduce.v2.app.job.TaskAttempt;
 import org.apache.hadoop.mapreduce.v2.util.MRApps;
 import org.apache.hadoop.yarn.util.ConverterUtils;
 import org.apache.hadoop.yarn.webapp.GenericExceptionHandler;
+import org.apache.hadoop.yarn.webapp.GuiceServletConfig;
 import org.apache.hadoop.yarn.webapp.WebServicesTestUtils;
 import org.codehaus.jettison.json.JSONArray;
 import org.codehaus.jettison.json.JSONException;
@@ -56,8 +58,6 @@ import org.w3c.dom.NodeList;
 import org.xml.sax.InputSource;
 
 import com.google.inject.Guice;
-import com.google.inject.Injector;
-import com.google.inject.servlet.GuiceServletContextListener;
 import com.google.inject.servlet.ServletModule;
 import com.sun.jersey.api.client.ClientResponse;
 import com.sun.jersey.api.client.UniformInterfaceException;
@@ -80,10 +80,10 @@ public class TestAMWebServicesAttempts extends JerseyTest {
   private static Configuration conf = new Configuration();
   private static AppContext appContext;
 
-  private Injector injector = Guice.createInjector(new ServletModule() {
+
+  private static class WebServletModule extends ServletModule {
     @Override
     protected void configureServlets() {
-
       appContext = new MockAppContext(0, 1, 2, 1);
       bind(JAXBContextResolver.class);
       bind(AMWebServices.class);
@@ -93,21 +93,19 @@ public class TestAMWebServicesAttempts extends JerseyTest {
 
       serve("/*").with(GuiceContainer.class);
     }
-  });
-
-  public class GuiceServletConfig extends GuiceServletContextListener {
+  }
 
-    @Override
-    protected Injector getInjector() {
-      return injector;
-    }
+  static {
+    GuiceServletConfig.setInjector(
+        Guice.createInjector(new WebServletModule()));
   }
 
   @Before
   @Override
   public void setUp() throws Exception {
     super.setUp();
-
+    GuiceServletConfig.setInjector(
+        Guice.createInjector(new WebServletModule()));
   }
 
   public TestAMWebServicesAttempts() {
@@ -379,7 +377,7 @@ public class TestAMWebServicesAttempts extends JerseyTest {
           fail("should have thrown exception on invalid uri");
         } catch (UniformInterfaceException ue) {
           ClientResponse response = ue.getResponse();
-          assertEquals(Status.NOT_FOUND, response.getClientResponseStatus());
+          assertResponseStatusCode(Status.NOT_FOUND, response.getStatusInfo());
           assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
           JSONObject msg = response.getEntity(JSONObject.class);
           JSONObject exception = msg.getJSONObject("RemoteException");

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5d58858b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/webapp/TestAMWebServicesJobConf.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/webapp/TestAMWebServicesJobConf.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/webapp/TestAMWebServicesJobConf.java
index 91e6615..a7958a9 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/webapp/TestAMWebServicesJobConf.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/webapp/TestAMWebServicesJobConf.java
@@ -40,16 +40,11 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.mapreduce.MRJobConfig;
 import org.apache.hadoop.mapreduce.v2.api.records.JobId;
 import org.apache.hadoop.mapreduce.v2.app.AppContext;
-import org.apache.hadoop.mapreduce.v2.app.ClusterInfo;
 import org.apache.hadoop.mapreduce.v2.app.MockAppContext;
-import org.apache.hadoop.mapreduce.v2.app.MockJobs;
 import org.apache.hadoop.mapreduce.v2.app.job.Job;
 import org.apache.hadoop.mapreduce.v2.util.MRApps;
-import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
-import org.apache.hadoop.yarn.api.records.ApplicationId;
-import org.apache.hadoop.yarn.event.EventHandler;
-import org.apache.hadoop.yarn.util.Clock;
 import org.apache.hadoop.yarn.webapp.GenericExceptionHandler;
+import org.apache.hadoop.yarn.webapp.GuiceServletConfig;
 import org.apache.hadoop.yarn.webapp.WebServicesTestUtils;
 import org.codehaus.jettison.json.JSONArray;
 import org.codehaus.jettison.json.JSONException;
@@ -62,10 +57,7 @@ import org.w3c.dom.Element;
 import org.w3c.dom.NodeList;
 import org.xml.sax.InputSource;
 
-import com.google.common.collect.Maps;
 import com.google.inject.Guice;
-import com.google.inject.Injector;
-import com.google.inject.servlet.GuiceServletContextListener;
 import com.google.inject.servlet.ServletModule;
 import com.sun.jersey.api.client.ClientResponse;
 import com.sun.jersey.api.client.WebResource;
@@ -87,7 +79,8 @@ public class TestAMWebServicesJobConf extends JerseyTest {
   private static File testConfDir = new File("target",
       TestAMWebServicesJobConf.class.getSimpleName() + "confDir");
 
-  private Injector injector = Guice.createInjector(new ServletModule() {
+  private static class WebServletModule extends ServletModule {
+
     @Override
     protected void configureServlets() {
 
@@ -124,14 +117,11 @@ public class TestAMWebServicesJobConf extends JerseyTest {
 
       serve("/*").with(GuiceContainer.class);
     }
-  });
-
-  public class GuiceServletConfig extends GuiceServletContextListener {
+  };
 
-    @Override
-    protected Injector getInjector() {
-      return injector;
-    }
+  static {
+    GuiceServletConfig.setInjector(
+        Guice.createInjector(new WebServletModule()));
   }
 
   @Before
@@ -139,7 +129,8 @@ public class TestAMWebServicesJobConf extends JerseyTest {
   public void setUp() throws Exception {
     super.setUp();
     testConfDir.mkdir();
-
+    GuiceServletConfig.setInjector(
+        Guice.createInjector(new WebServletModule()));
   }
 
   @AfterClass

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5d58858b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/webapp/TestAMWebServicesJobs.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/webapp/TestAMWebServicesJobs.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/webapp/TestAMWebServicesJobs.java
index a61e407..128bb5a 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/webapp/TestAMWebServicesJobs.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/webapp/TestAMWebServicesJobs.java
@@ -19,6 +19,7 @@
 package org.apache.hadoop.mapreduce.v2.app.webapp;
 
 import static org.apache.hadoop.yarn.util.StringHelper.ujoin;
+import static org.apache.hadoop.yarn.webapp.WebServicesTestUtils.assertResponseStatusCode;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertTrue;
@@ -45,6 +46,7 @@ import org.apache.hadoop.security.authorize.AccessControlList;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.util.Times;
 import org.apache.hadoop.yarn.webapp.GenericExceptionHandler;
+import org.apache.hadoop.yarn.webapp.GuiceServletConfig;
 import org.apache.hadoop.yarn.webapp.WebServicesTestUtils;
 import org.codehaus.jettison.json.JSONArray;
 import org.codehaus.jettison.json.JSONException;
@@ -57,8 +59,6 @@ import org.w3c.dom.NodeList;
 import org.xml.sax.InputSource;
 
 import com.google.inject.Guice;
-import com.google.inject.Injector;
-import com.google.inject.servlet.GuiceServletContextListener;
 import com.google.inject.servlet.ServletModule;
 import com.sun.jersey.api.client.ClientResponse;
 import com.sun.jersey.api.client.ClientResponse.Status;
@@ -82,10 +82,9 @@ public class TestAMWebServicesJobs extends JerseyTest {
   private static Configuration conf = new Configuration();
   private static AppContext appContext;
 
-  private Injector injector = Guice.createInjector(new ServletModule() {
+  private static class WebServletModule extends ServletModule {
     @Override
     protected void configureServlets() {
-
       appContext = new MockAppContext(0, 1, 2, 1);
       bind(JAXBContextResolver.class);
       bind(AMWebServices.class);
@@ -95,21 +94,19 @@ public class TestAMWebServicesJobs extends JerseyTest {
 
       serve("/*").with(GuiceContainer.class);
     }
-  });
-
-  public class GuiceServletConfig extends GuiceServletContextListener {
+  }
 
-    @Override
-    protected Injector getInjector() {
-      return injector;
-    }
+  static {
+    GuiceServletConfig.setInjector(
+        Guice.createInjector(new WebServletModule()));
   }
 
   @Before
   @Override
   public void setUp() throws Exception {
     super.setUp();
-
+    GuiceServletConfig.setInjector(
+        Guice.createInjector(new WebServletModule()));
   }
 
   public TestAMWebServicesJobs() {
@@ -256,7 +253,7 @@ public class TestAMWebServicesJobs extends JerseyTest {
       fail("should have thrown exception on invalid uri");
     } catch (UniformInterfaceException ue) {
       ClientResponse response = ue.getResponse();
-      assertEquals(Status.NOT_FOUND, response.getClientResponseStatus());
+      assertResponseStatusCode(Status.NOT_FOUND, response.getStatusInfo());
       assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
       JSONObject msg = response.getEntity(JSONObject.class);
       JSONObject exception = msg.getJSONObject("RemoteException");
@@ -283,7 +280,7 @@ public class TestAMWebServicesJobs extends JerseyTest {
       fail("should have thrown exception on invalid uri");
     } catch (UniformInterfaceException ue) {
       ClientResponse response = ue.getResponse();
-      assertEquals(Status.NOT_FOUND, response.getClientResponseStatus());
+      assertResponseStatusCode(Status.NOT_FOUND, response.getStatusInfo());
       assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
       JSONObject msg = response.getEntity(JSONObject.class);
       JSONObject exception = msg.getJSONObject("RemoteException");
@@ -306,7 +303,7 @@ public class TestAMWebServicesJobs extends JerseyTest {
       fail("should have thrown exception on invalid uri");
     } catch (UniformInterfaceException ue) {
       ClientResponse response = ue.getResponse();
-      assertEquals(Status.NOT_FOUND, response.getClientResponseStatus());
+      assertResponseStatusCode(Status.NOT_FOUND, response.getStatusInfo());
       assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
       JSONObject msg = response.getEntity(JSONObject.class);
       JSONObject exception = msg.getJSONObject("RemoteException");
@@ -329,7 +326,7 @@ public class TestAMWebServicesJobs extends JerseyTest {
       fail("should have thrown exception on invalid uri");
     } catch (UniformInterfaceException ue) {
       ClientResponse response = ue.getResponse();
-      assertEquals(Status.NOT_FOUND, response.getClientResponseStatus());
+      assertResponseStatusCode(Status.NOT_FOUND, response.getStatusInfo());
       assertEquals(MediaType.APPLICATION_XML_TYPE, response.getType());
       String msg = response.getEntity(String.class);
       System.out.println(msg);
@@ -368,7 +365,7 @@ public class TestAMWebServicesJobs extends JerseyTest {
       fail("should have thrown exception on invalid uri");
     } catch (UniformInterfaceException ue) {
       ClientResponse response = ue.getResponse();
-      assertEquals(Status.NOT_FOUND, response.getClientResponseStatus());
+      assertResponseStatusCode(Status.NOT_FOUND, response.getStatusInfo());
       assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
       JSONObject msg = response.getEntity(JSONObject.class);
       JSONObject exception = msg.getJSONObject("RemoteException");

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5d58858b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/webapp/TestAMWebServicesTasks.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/webapp/TestAMWebServicesTasks.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/webapp/TestAMWebServicesTasks.java
index 15ef5ac..b9bcc01 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/webapp/TestAMWebServicesTasks.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/webapp/TestAMWebServicesTasks.java
@@ -18,6 +18,7 @@
 
 package org.apache.hadoop.mapreduce.v2.app.webapp;
 
+import static org.apache.hadoop.yarn.webapp.WebServicesTestUtils.assertResponseStatusCode;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertTrue;
@@ -41,6 +42,7 @@ import org.apache.hadoop.mapreduce.v2.app.job.Job;
 import org.apache.hadoop.mapreduce.v2.app.job.Task;
 import org.apache.hadoop.mapreduce.v2.util.MRApps;
 import org.apache.hadoop.yarn.webapp.GenericExceptionHandler;
+import org.apache.hadoop.yarn.webapp.GuiceServletConfig;
 import org.apache.hadoop.yarn.webapp.WebServicesTestUtils;
 import org.codehaus.jettison.json.JSONArray;
 import org.codehaus.jettison.json.JSONException;
@@ -53,8 +55,6 @@ import org.w3c.dom.NodeList;
 import org.xml.sax.InputSource;
 
 import com.google.inject.Guice;
-import com.google.inject.Injector;
-import com.google.inject.servlet.GuiceServletContextListener;
 import com.google.inject.servlet.ServletModule;
 import com.sun.jersey.api.client.ClientResponse;
 import com.sun.jersey.api.client.ClientResponse.Status;
@@ -77,7 +77,7 @@ public class TestAMWebServicesTasks extends JerseyTest {
   private static Configuration conf = new Configuration();
   private static AppContext appContext;
 
-  private Injector injector = Guice.createInjector(new ServletModule() {
+  private static class WebServletModule extends ServletModule {
     @Override
     protected void configureServlets() {
 
@@ -90,21 +90,19 @@ public class TestAMWebServicesTasks extends JerseyTest {
 
       serve("/*").with(GuiceContainer.class);
     }
-  });
-
-  public class GuiceServletConfig extends GuiceServletContextListener {
+  }
 
-    @Override
-    protected Injector getInjector() {
-      return injector;
-    }
+  static {
+    GuiceServletConfig.setInjector(
+        Guice.createInjector(new WebServletModule()));
   }
 
   @Before
   @Override
   public void setUp() throws Exception {
     super.setUp();
-
+    GuiceServletConfig.setInjector(
+        Guice.createInjector(new WebServletModule()));
   }
 
   public TestAMWebServicesTasks() {
@@ -254,7 +252,7 @@ public class TestAMWebServicesTasks extends JerseyTest {
         fail("should have thrown exception on invalid uri");
       } catch (UniformInterfaceException ue) {
         ClientResponse response = ue.getResponse();
-        assertEquals(Status.BAD_REQUEST, response.getClientResponseStatus());
+        assertResponseStatusCode(Status.BAD_REQUEST, response.getStatusInfo());
         assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
         JSONObject msg = response.getEntity(JSONObject.class);
         JSONObject exception = msg.getJSONObject("RemoteException");
@@ -348,7 +346,7 @@ public class TestAMWebServicesTasks extends JerseyTest {
         fail("should have thrown exception on invalid uri");
       } catch (UniformInterfaceException ue) {
         ClientResponse response = ue.getResponse();
-        assertEquals(Status.NOT_FOUND, response.getClientResponseStatus());
+        assertResponseStatusCode(Status.NOT_FOUND, response.getStatusInfo());
         assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
         JSONObject msg = response.getEntity(JSONObject.class);
         JSONObject exception = msg.getJSONObject("RemoteException");
@@ -382,7 +380,7 @@ public class TestAMWebServicesTasks extends JerseyTest {
         fail("should have thrown exception on invalid uri");
       } catch (UniformInterfaceException ue) {
         ClientResponse response = ue.getResponse();
-        assertEquals(Status.NOT_FOUND, response.getClientResponseStatus());
+        assertResponseStatusCode(Status.NOT_FOUND, response.getStatusInfo());
         assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
         JSONObject msg = response.getEntity(JSONObject.class);
         JSONObject exception = msg.getJSONObject("RemoteException");
@@ -414,7 +412,7 @@ public class TestAMWebServicesTasks extends JerseyTest {
         fail("should have thrown exception on invalid uri");
       } catch (UniformInterfaceException ue) {
         ClientResponse response = ue.getResponse();
-        assertEquals(Status.NOT_FOUND, response.getClientResponseStatus());
+        assertResponseStatusCode(Status.NOT_FOUND, response.getStatusInfo());
         assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
         JSONObject msg = response.getEntity(JSONObject.class);
         JSONObject exception = msg.getJSONObject("RemoteException");
@@ -448,7 +446,7 @@ public class TestAMWebServicesTasks extends JerseyTest {
         fail("should have thrown exception on invalid uri");
       } catch (UniformInterfaceException ue) {
         ClientResponse response = ue.getResponse();
-        assertEquals(Status.NOT_FOUND, response.getClientResponseStatus());
+        assertResponseStatusCode(Status.NOT_FOUND, response.getStatusInfo());
         assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
         JSONObject msg = response.getEntity(JSONObject.class);
         JSONObject exception = msg.getJSONObject("RemoteException");
@@ -482,7 +480,7 @@ public class TestAMWebServicesTasks extends JerseyTest {
         fail("should have thrown exception on invalid uri");
       } catch (UniformInterfaceException ue) {
         ClientResponse response = ue.getResponse();
-        assertEquals(Status.NOT_FOUND, response.getClientResponseStatus());
+        assertResponseStatusCode(Status.NOT_FOUND, response.getStatusInfo());
         assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
         JSONObject msg = response.getEntity(JSONObject.class);
         JSONObject exception = msg.getJSONObject("RemoteException");

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5d58858b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/test/java/org/apache/hadoop/mapreduce/v2/hs/webapp/TestHsWebServices.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/test/java/org/apache/hadoop/mapreduce/v2/hs/webapp/TestHsWebServices.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/test/java/org/apache/hadoop/mapreduce/v2/hs/webapp/TestHsWebServices.java
index 138554d..b9b45af 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/test/java/org/apache/hadoop/mapreduce/v2/hs/webapp/TestHsWebServices.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/test/java/org/apache/hadoop/mapreduce/v2/hs/webapp/TestHsWebServices.java
@@ -18,6 +18,7 @@
 
 package org.apache.hadoop.mapreduce.v2.hs.webapp;
 
+import static org.apache.hadoop.yarn.webapp.WebServicesTestUtils.assertResponseStatusCode;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.fail;
 
@@ -35,6 +36,7 @@ import org.apache.hadoop.mapreduce.v2.hs.JobHistoryServer;
 import org.apache.hadoop.mapreduce.v2.hs.MockHistoryContext;
 import org.apache.hadoop.util.VersionInfo;
 import org.apache.hadoop.yarn.webapp.GenericExceptionHandler;
+import org.apache.hadoop.yarn.webapp.GuiceServletConfig;
 import org.apache.hadoop.yarn.webapp.WebApp;
 import org.apache.hadoop.yarn.webapp.WebServicesTestUtils;
 import org.codehaus.jettison.json.JSONException;
@@ -47,8 +49,6 @@ import org.w3c.dom.NodeList;
 import org.xml.sax.InputSource;
 
 import com.google.inject.Guice;
-import com.google.inject.Injector;
-import com.google.inject.servlet.GuiceServletContextListener;
 import com.google.inject.servlet.ServletModule;
 import com.sun.jersey.api.client.ClientResponse;
 import com.sun.jersey.api.client.ClientResponse.Status;
@@ -70,10 +70,9 @@ public class TestHsWebServices extends JerseyTest {
   private static HistoryContext appContext;
   private static HsWebApp webApp;
 
-  private Injector injector = Guice.createInjector(new ServletModule() {
+  private static class WebServletModule extends ServletModule {
     @Override
     protected void configureServlets() {
-
       appContext = new MockHistoryContext(0, 1, 1, 1);
       JobHistory jobHistoryService = new JobHistory();
       HistoryContext historyContext = (HistoryContext) jobHistoryService;
@@ -89,20 +88,19 @@ public class TestHsWebServices extends JerseyTest {
 
       serve("/*").with(GuiceContainer.class);
     }
-  });
-
-  public class GuiceServletConfig extends GuiceServletContextListener {
+  }
 
-    @Override
-    protected Injector getInjector() {
-      return injector;
-    }
+  static {
+    GuiceServletConfig.setInjector(
+        Guice.createInjector(new WebServletModule()));
   }
 
   @Before
   @Override
   public void setUp() throws Exception {
     super.setUp();
+    GuiceServletConfig.setInjector(
+        Guice.createInjector(new WebServletModule()));
   }
 
   public TestHsWebServices() {
@@ -212,7 +210,7 @@ public class TestHsWebServices extends JerseyTest {
       fail("should have thrown exception on invalid uri");
     } catch (UniformInterfaceException ue) {
       ClientResponse response = ue.getResponse();
-      assertEquals(Status.NOT_FOUND, response.getClientResponseStatus());
+      assertResponseStatusCode(Status.NOT_FOUND, response.getStatusInfo());
       WebServicesTestUtils.checkStringMatch(
           "error string exists and shouldn't", "", responseStr);
     }
@@ -228,7 +226,7 @@ public class TestHsWebServices extends JerseyTest {
       fail("should have thrown exception on invalid uri");
     } catch (UniformInterfaceException ue) {
       ClientResponse response = ue.getResponse();
-      assertEquals(Status.NOT_FOUND, response.getClientResponseStatus());
+      assertResponseStatusCode(Status.NOT_FOUND, response.getStatusInfo());
       WebServicesTestUtils.checkStringMatch(
           "error string exists and shouldn't", "", responseStr);
     }
@@ -244,8 +242,8 @@ public class TestHsWebServices extends JerseyTest {
       fail("should have thrown exception on invalid uri");
     } catch (UniformInterfaceException ue) {
       ClientResponse response = ue.getResponse();
-      assertEquals(Status.INTERNAL_SERVER_ERROR,
-          response.getClientResponseStatus());
+      assertResponseStatusCode(Status.INTERNAL_SERVER_ERROR,
+          response.getStatusInfo());
       WebServicesTestUtils.checkStringMatch(
           "error string exists and shouldn't", "", responseStr);
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5d58858b/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 54c2792..f7b137f 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
@@ -18,6 +18,7 @@
 
 package org.apache.hadoop.mapreduce.v2.hs.webapp;
 
+import static org.apache.hadoop.yarn.webapp.WebServicesTestUtils.assertResponseStatusCode;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertTrue;
@@ -46,6 +47,7 @@ import org.apache.hadoop.mapreduce.v2.hs.MockHistoryContext;
 import org.apache.hadoop.mapreduce.v2.util.MRApps;
 import org.apache.hadoop.yarn.util.ConverterUtils;
 import org.apache.hadoop.yarn.webapp.GenericExceptionHandler;
+import org.apache.hadoop.yarn.webapp.GuiceServletConfig;
 import org.apache.hadoop.yarn.webapp.WebApp;
 import org.apache.hadoop.yarn.webapp.WebServicesTestUtils;
 import org.codehaus.jettison.json.JSONArray;
@@ -59,8 +61,6 @@ import org.w3c.dom.NodeList;
 import org.xml.sax.InputSource;
 
 import com.google.inject.Guice;
-import com.google.inject.Injector;
-import com.google.inject.servlet.GuiceServletContextListener;
 import com.google.inject.servlet.ServletModule;
 import com.sun.jersey.api.client.ClientResponse;
 import com.sun.jersey.api.client.ClientResponse.Status;
@@ -85,10 +85,9 @@ public class TestHsWebServicesAttempts extends JerseyTest {
   private static HistoryContext appContext;
   private static HsWebApp webApp;
 
-  private Injector injector = Guice.createInjector(new ServletModule() {
+  private static class WebServletModule extends ServletModule {
     @Override
     protected void configureServlets() {
-
       appContext = new MockHistoryContext(0, 1, 2, 1);
       webApp = mock(HsWebApp.class);
       when(webApp.name()).thenReturn("hsmockwebapp");
@@ -103,29 +102,27 @@ public class TestHsWebServicesAttempts extends JerseyTest {
 
       serve("/*").with(GuiceContainer.class);
     }
-  });
-
-  public class GuiceServletConfig extends GuiceServletContextListener {
+  }
 
-    @Override
-    protected Injector getInjector() {
-      return injector;
-    }
+  static {
+    GuiceServletConfig.setInjector(
+        Guice.createInjector(new WebServletModule()));
   }
 
   @Before
   @Override
   public void setUp() throws Exception {
     super.setUp();
-
+    GuiceServletConfig.setInjector(
+        Guice.createInjector(new WebServletModule()));
   }
 
   public TestHsWebServicesAttempts() {
-    super(new WebAppDescriptor.Builder(
-        "org.apache.hadoop.mapreduce.v2.hs.webapp")
-        .contextListenerClass(GuiceServletConfig.class)
-        .filterClass(com.google.inject.servlet.GuiceFilter.class)
-        .contextPath("jersey-guice-filter").servletPath("/").build());
+    super(
+        new WebAppDescriptor.Builder("org.apache.hadoop.mapreduce.v2.hs.webapp")
+            .contextListenerClass(GuiceServletConfig.class)
+            .filterClass(com.google.inject.servlet.GuiceFilter.class)
+            .contextPath("jersey-guice-filter").servletPath("/").build());
   }
 
   @Test
@@ -397,7 +394,7 @@ public class TestHsWebServicesAttempts extends JerseyTest {
           fail("should have thrown exception on invalid uri");
         } catch (UniformInterfaceException ue) {
           ClientResponse response = ue.getResponse();
-          assertEquals(Status.NOT_FOUND, response.getClientResponseStatus());
+          assertResponseStatusCode(Status.NOT_FOUND, response.getStatusInfo());
           assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
           JSONObject msg = response.getEntity(JSONObject.class);
           JSONObject exception = msg.getJSONObject("RemoteException");

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5d58858b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/test/java/org/apache/hadoop/mapreduce/v2/hs/webapp/TestHsWebServicesJobConf.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/test/java/org/apache/hadoop/mapreduce/v2/hs/webapp/TestHsWebServicesJobConf.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/test/java/org/apache/hadoop/mapreduce/v2/hs/webapp/TestHsWebServicesJobConf.java
index aeacb95..a222302 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/test/java/org/apache/hadoop/mapreduce/v2/hs/webapp/TestHsWebServicesJobConf.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/test/java/org/apache/hadoop/mapreduce/v2/hs/webapp/TestHsWebServicesJobConf.java
@@ -47,6 +47,7 @@ import org.apache.hadoop.mapreduce.v2.hs.HistoryContext;
 import org.apache.hadoop.mapreduce.v2.hs.MockHistoryContext;
 import org.apache.hadoop.mapreduce.v2.util.MRApps;
 import org.apache.hadoop.yarn.webapp.GenericExceptionHandler;
+import org.apache.hadoop.yarn.webapp.GuiceServletConfig;
 import org.apache.hadoop.yarn.webapp.WebApp;
 import org.apache.hadoop.yarn.webapp.WebServicesTestUtils;
 import org.codehaus.jettison.json.JSONArray;
@@ -61,8 +62,6 @@ import org.w3c.dom.NodeList;
 import org.xml.sax.InputSource;
 
 import com.google.inject.Guice;
-import com.google.inject.Injector;
-import com.google.inject.servlet.GuiceServletContextListener;
 import com.google.inject.servlet.ServletModule;
 import com.sun.jersey.api.client.ClientResponse;
 import com.sun.jersey.api.client.WebResource;
@@ -85,7 +84,8 @@ public class TestHsWebServicesJobConf extends JerseyTest {
   private static File testConfDir = new File("target",
       TestHsWebServicesJobConf.class.getSimpleName() + "confDir");
 
-  private Injector injector = Guice.createInjector(new ServletModule() {
+  private static class WebServletModule extends ServletModule {
+
     @Override
     protected void configureServlets() {
 
@@ -127,14 +127,11 @@ public class TestHsWebServicesJobConf extends JerseyTest {
 
       serve("/*").with(GuiceContainer.class);
     }
-  });
-
-  public class GuiceServletConfig extends GuiceServletContextListener {
+  };
 
-    @Override
-    protected Injector getInjector() {
-      return injector;
-    }
+  static {
+    GuiceServletConfig.setInjector(
+        Guice.createInjector(new WebServletModule()));
   }
 
   @Before
@@ -142,7 +139,8 @@ public class TestHsWebServicesJobConf extends JerseyTest {
   public void setUp() throws Exception {
     super.setUp();
     testConfDir.mkdir();
-
+    GuiceServletConfig.setInjector(
+        Guice.createInjector(new WebServletModule()));
   }
 
   @AfterClass

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5d58858b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/test/java/org/apache/hadoop/mapreduce/v2/hs/webapp/TestHsWebServicesJobs.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/test/java/org/apache/hadoop/mapreduce/v2/hs/webapp/TestHsWebServicesJobs.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/test/java/org/apache/hadoop/mapreduce/v2/hs/webapp/TestHsWebServicesJobs.java
index 4c4088f..b2df3c6 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/test/java/org/apache/hadoop/mapreduce/v2/hs/webapp/TestHsWebServicesJobs.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/test/java/org/apache/hadoop/mapreduce/v2/hs/webapp/TestHsWebServicesJobs.java
@@ -20,6 +20,7 @@ package org.apache.hadoop.mapreduce.v2.hs.webapp;
 
 import static org.apache.hadoop.yarn.util.StringHelper.join;
 import static org.apache.hadoop.yarn.util.StringHelper.ujoin;
+import static org.apache.hadoop.yarn.webapp.WebServicesTestUtils.assertResponseStatusCode;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertTrue;
@@ -44,6 +45,7 @@ import org.apache.hadoop.mapreduce.v2.hs.MockHistoryContext;
 import org.apache.hadoop.mapreduce.v2.util.MRApps;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.webapp.GenericExceptionHandler;
+import org.apache.hadoop.yarn.webapp.GuiceServletConfig;
 import org.apache.hadoop.yarn.webapp.WebApp;
 import org.apache.hadoop.yarn.webapp.WebServicesTestUtils;
 import org.codehaus.jettison.json.JSONArray;
@@ -57,8 +59,6 @@ import org.w3c.dom.NodeList;
 import org.xml.sax.InputSource;
 
 import com.google.inject.Guice;
-import com.google.inject.Injector;
-import com.google.inject.servlet.GuiceServletContextListener;
 import com.google.inject.servlet.ServletModule;
 import com.sun.jersey.api.client.ClientResponse;
 import com.sun.jersey.api.client.ClientResponse.Status;
@@ -82,10 +82,9 @@ public class TestHsWebServicesJobs extends JerseyTest {
   private static MockHistoryContext appContext;
   private static HsWebApp webApp;
 
-  private Injector injector = Guice.createInjector(new ServletModule() {
+  private static class WebServletModule extends ServletModule {
     @Override
     protected void configureServlets() {
-
       appContext = new MockHistoryContext(0, 1, 2, 1, false);
       webApp = mock(HsWebApp.class);
       when(webApp.name()).thenReturn("hsmockwebapp");
@@ -100,21 +99,19 @@ public class TestHsWebServicesJobs extends JerseyTest {
 
       serve("/*").with(GuiceContainer.class);
     }
-  });
-
-  public class GuiceServletConfig extends GuiceServletContextListener {
+  }
 
-    @Override
-    protected Injector getInjector() {
-      return injector;
-    }
+  static {
+    GuiceServletConfig.setInjector(
+        Guice.createInjector(new WebServletModule()));
   }
 
   @Before
   @Override
   public void setUp() throws Exception {
     super.setUp();
-
+    GuiceServletConfig.setInjector(
+        Guice.createInjector(new WebServletModule()));
   }
 
   public TestHsWebServicesJobs() {
@@ -331,7 +328,7 @@ public class TestHsWebServicesJobs extends JerseyTest {
       fail("should have thrown exception on invalid uri");
     } catch (UniformInterfaceException ue) {
       ClientResponse response = ue.getResponse();
-      assertEquals(Status.NOT_FOUND, response.getClientResponseStatus());
+      assertResponseStatusCode(Status.NOT_FOUND, response.getStatusInfo());
       assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
       JSONObject msg = response.getEntity(JSONObject.class);
       JSONObject exception = msg.getJSONObject("RemoteException");
@@ -359,7 +356,7 @@ public class TestHsWebServicesJobs extends JerseyTest {
       fail("should have thrown exception on invalid uri");
     } catch (UniformInterfaceException ue) {
       ClientResponse response = ue.getResponse();
-      assertEquals(Status.NOT_FOUND, response.getClientResponseStatus());
+      assertResponseStatusCode(Status.NOT_FOUND, response.getStatusInfo());
       assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
       JSONObject msg = response.getEntity(JSONObject.class);
       JSONObject exception = msg.getJSONObject("RemoteException");
@@ -383,7 +380,7 @@ public class TestHsWebServicesJobs extends JerseyTest {
       fail("should have thrown exception on invalid uri");
     } catch (UniformInterfaceException ue) {
       ClientResponse response = ue.getResponse();
-      assertEquals(Status.NOT_FOUND, response.getClientResponseStatus());
+      assertResponseStatusCode(Status.NOT_FOUND, response.getStatusInfo());
       assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
       JSONObject msg = response.getEntity(JSONObject.class);
       JSONObject exception = msg.getJSONObject("RemoteException");
@@ -407,7 +404,7 @@ public class TestHsWebServicesJobs extends JerseyTest {
       fail("should have thrown exception on invalid uri");
     } catch (UniformInterfaceException ue) {
       ClientResponse response = ue.getResponse();
-      assertEquals(Status.NOT_FOUND, response.getClientResponseStatus());
+      assertResponseStatusCode(Status.NOT_FOUND, response.getStatusInfo());
       assertEquals(MediaType.APPLICATION_XML_TYPE, response.getType());
       String msg = response.getEntity(String.class);
       System.out.println(msg);
@@ -446,7 +443,7 @@ public class TestHsWebServicesJobs extends JerseyTest {
       fail("should have thrown exception on invalid uri");
     } catch (UniformInterfaceException ue) {
       ClientResponse response = ue.getResponse();
-      assertEquals(Status.NOT_FOUND, response.getClientResponseStatus());
+      assertResponseStatusCode(Status.NOT_FOUND, response.getStatusInfo());
       assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
       JSONObject msg = response.getEntity(JSONObject.class);
       JSONObject exception = msg.getJSONObject("RemoteException");
@@ -527,7 +524,7 @@ public class TestHsWebServicesJobs extends JerseyTest {
   public void testJobCountersForKilledJob() throws Exception {
     WebResource r = resource();
     appContext = new MockHistoryContext(0, 1, 1, 1, true);
-    injector = Guice.createInjector(new ServletModule() {
+    GuiceServletConfig.setInjector(Guice.createInjector(new ServletModule() {
       @Override
       protected void configureServlets() {
 
@@ -544,7 +541,7 @@ public class TestHsWebServicesJobs extends JerseyTest {
 
         serve("/*").with(GuiceContainer.class);
       }
-    });
+    }));
     
     Map<JobId, Job> jobsMap = appContext.getAllJobs();
     for (JobId id : jobsMap.keySet()) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5d58858b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/test/java/org/apache/hadoop/mapreduce/v2/hs/webapp/TestHsWebServicesJobsQuery.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/test/java/org/apache/hadoop/mapreduce/v2/hs/webapp/TestHsWebServicesJobsQuery.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/test/java/org/apache/hadoop/mapreduce/v2/hs/webapp/TestHsWebServicesJobsQuery.java
index 8891ec7..343fa6d 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/test/java/org/apache/hadoop/mapreduce/v2/hs/webapp/TestHsWebServicesJobsQuery.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/test/java/org/apache/hadoop/mapreduce/v2/hs/webapp/TestHsWebServicesJobsQuery.java
@@ -18,6 +18,7 @@
 
 package org.apache.hadoop.mapreduce.v2.hs.webapp;
 
+import static org.apache.hadoop.yarn.webapp.WebServicesTestUtils.assertResponseStatusCode;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 import static org.mockito.Mockito.mock;
@@ -39,6 +40,7 @@ import org.apache.hadoop.mapreduce.v2.hs.HistoryContext;
 import org.apache.hadoop.mapreduce.v2.hs.MockHistoryContext;
 import org.apache.hadoop.mapreduce.v2.util.MRApps;
 import org.apache.hadoop.yarn.webapp.GenericExceptionHandler;
+import org.apache.hadoop.yarn.webapp.GuiceServletConfig;
 import org.apache.hadoop.yarn.webapp.WebApp;
 import org.apache.hadoop.yarn.webapp.WebServicesTestUtils;
 import org.codehaus.jettison.json.JSONArray;
@@ -48,8 +50,6 @@ import org.junit.Before;
 import org.junit.Test;
 
 import com.google.inject.Guice;
-import com.google.inject.Injector;
-import com.google.inject.servlet.GuiceServletContextListener;
 import com.google.inject.servlet.ServletModule;
 import com.sun.jersey.api.client.ClientResponse;
 import com.sun.jersey.api.client.ClientResponse.Status;
@@ -70,10 +70,9 @@ public class TestHsWebServicesJobsQuery extends JerseyTest {
   private static MockHistoryContext appContext;
   private static HsWebApp webApp;
 
-  private Injector injector = Guice.createInjector(new ServletModule() {
+  private static class WebServletModule extends ServletModule {
     @Override
     protected void configureServlets() {
-
       appContext = new MockHistoryContext(3, 2, 1);
       webApp = mock(HsWebApp.class);
       when(webApp.name()).thenReturn("hsmockwebapp");
@@ -88,21 +87,19 @@ public class TestHsWebServicesJobsQuery extends JerseyTest {
 
       serve("/*").with(GuiceContainer.class);
     }
-  });
-
-  public class GuiceServletConfig extends GuiceServletContextListener {
+  }
 
-    @Override
-    protected Injector getInjector() {
-      return injector;
-    }
+  static {
+    GuiceServletConfig.setInjector(
+        Guice.createInjector(new WebServletModule()));
   }
 
   @Before
   @Override
   public void setUp() throws Exception {
     super.setUp();
-
+    GuiceServletConfig.setInjector(
+        Guice.createInjector(new WebServletModule()));
   }
 
   public TestHsWebServicesJobsQuery() {
@@ -136,7 +133,8 @@ public class TestHsWebServicesJobsQuery extends JerseyTest {
     assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
     JSONObject json = response.getEntity(JSONObject.class);
     assertEquals("incorrect number of elements", 1, json.length());
-    assertEquals("jobs is not null", JSONObject.NULL, json.get("jobs"));
+    assertEquals("jobs is not empty",
+        new JSONObject().toString(), json.get("jobs").toString());
   }
 
   @Test
@@ -173,7 +171,7 @@ public class TestHsWebServicesJobsQuery extends JerseyTest {
         .path("mapreduce").path("jobs").queryParam("state", "InvalidState")
         .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
 
-    assertEquals(Status.BAD_REQUEST, response.getClientResponseStatus());
+    assertResponseStatusCode(Status.BAD_REQUEST, response.getStatusInfo());
     assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
     JSONObject msg = response.getEntity(JSONObject.class);
     JSONObject exception = msg.getJSONObject("RemoteException");
@@ -202,7 +200,8 @@ public class TestHsWebServicesJobsQuery extends JerseyTest {
     assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
     JSONObject json = response.getEntity(JSONObject.class);
     assertEquals("incorrect number of elements", 1, json.length());
-    assertEquals("jobs is not null", JSONObject.NULL, json.get("jobs"));
+    assertEquals("jobs is not empty",
+        new JSONObject().toString(), json.get("jobs").toString());
   }
 
   @Test
@@ -248,7 +247,7 @@ public class TestHsWebServicesJobsQuery extends JerseyTest {
         .path("mapreduce").path("jobs").queryParam("limit", "-1")
         .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
 
-    assertEquals(Status.BAD_REQUEST, response.getClientResponseStatus());
+    assertResponseStatusCode(Status.BAD_REQUEST, response.getStatusInfo());
     assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
     JSONObject msg = response.getEntity(JSONObject.class);
     JSONObject exception = msg.getJSONObject("RemoteException");
@@ -287,7 +286,8 @@ public class TestHsWebServicesJobsQuery extends JerseyTest {
     assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
     JSONObject json = response.getEntity(JSONObject.class);
     assertEquals("incorrect number of elements", 1, json.length());
-    assertEquals("jobs is not null", JSONObject.NULL, json.get("jobs"));
+    assertEquals("jobs is not empty",
+        new JSONObject().toString(), json.get("jobs").toString());
   }
 
   @Test
@@ -319,7 +319,8 @@ public class TestHsWebServicesJobsQuery extends JerseyTest {
     assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
     JSONObject json = response.getEntity(JSONObject.class);
     assertEquals("incorrect number of elements", 1, json.length());
-    assertEquals("jobs is not null", JSONObject.NULL, json.get("jobs"));
+    assertEquals("jobs is not empty",
+        new JSONObject().toString(), json.get("jobs").toString());
   }
 
   @Test
@@ -360,7 +361,7 @@ public class TestHsWebServicesJobsQuery extends JerseyTest {
         .queryParam("startedTimeBegin", String.valueOf(now))
         .queryParam("startedTimeEnd", String.valueOf(40000))
         .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
-    assertEquals(Status.BAD_REQUEST, response.getClientResponseStatus());
+    assertResponseStatusCode(Status.BAD_REQUEST, response.getStatusInfo());
     assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
     JSONObject msg = response.getEntity(JSONObject.class);
     JSONObject exception = msg.getJSONObject("RemoteException");
@@ -386,7 +387,7 @@ public class TestHsWebServicesJobsQuery extends JerseyTest {
     ClientResponse response = r.path("ws").path("v1").path("history")
         .path("mapreduce").path("jobs").queryParam("startedTimeBegin", "efsd")
         .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
-    assertEquals(Status.BAD_REQUEST, response.getClientResponseStatus());
+    assertResponseStatusCode(Status.BAD_REQUEST, response.getStatusInfo());
     assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
     JSONObject msg = response.getEntity(JSONObject.class);
     JSONObject exception = msg.getJSONObject("RemoteException");
@@ -412,7 +413,7 @@ public class TestHsWebServicesJobsQuery extends JerseyTest {
     ClientResponse response = r.path("ws").path("v1").path("history")
         .path("mapreduce").path("jobs").queryParam("startedTimeEnd", "efsd")
         .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
-    assertEquals(Status.BAD_REQUEST, response.getClientResponseStatus());
+    assertResponseStatusCode(Status.BAD_REQUEST, response.getStatusInfo());
     assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
     JSONObject msg = response.getEntity(JSONObject.class);
     JSONObject exception = msg.getJSONObject("RemoteException");
@@ -438,7 +439,7 @@ public class TestHsWebServicesJobsQuery extends JerseyTest {
         .path("mapreduce").path("jobs")
         .queryParam("startedTimeBegin", String.valueOf(-1000))
         .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
-    assertEquals(Status.BAD_REQUEST, response.getClientResponseStatus());
+    assertResponseStatusCode(Status.BAD_REQUEST, response.getStatusInfo());
     assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
     JSONObject msg = response.getEntity(JSONObject.class);
     JSONObject exception = msg.getJSONObject("RemoteException");
@@ -464,7 +465,7 @@ public class TestHsWebServicesJobsQuery extends JerseyTest {
         .path("mapreduce").path("jobs")
         .queryParam("startedTimeEnd", String.valueOf(-1000))
         .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
-    assertEquals(Status.BAD_REQUEST, response.getClientResponseStatus());
+    assertResponseStatusCode(Status.BAD_REQUEST, response.getStatusInfo());
     assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
     JSONObject msg = response.getEntity(JSONObject.class);
     JSONObject exception = msg.getJSONObject("RemoteException");
@@ -488,7 +489,7 @@ public class TestHsWebServicesJobsQuery extends JerseyTest {
         .path("mapreduce").path("jobs")
         .queryParam("finishedTimeEnd", String.valueOf(-1000))
         .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
-    assertEquals(Status.BAD_REQUEST, response.getClientResponseStatus());
+    assertResponseStatusCode(Status.BAD_REQUEST, response.getStatusInfo());
     assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
     JSONObject msg = response.getEntity(JSONObject.class);
     JSONObject exception = msg.getJSONObject("RemoteException");
@@ -512,7 +513,7 @@ public class TestHsWebServicesJobsQuery extends JerseyTest {
         .path("mapreduce").path("jobs")
         .queryParam("finishedTimeBegin", String.valueOf(-1000))
         .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
-    assertEquals(Status.BAD_REQUEST, response.getClientResponseStatus());
+    assertResponseStatusCode(Status.BAD_REQUEST, response.getStatusInfo());
     assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
     JSONObject msg = response.getEntity(JSONObject.class);
     JSONObject exception = msg.getJSONObject("RemoteException");
@@ -539,7 +540,7 @@ public class TestHsWebServicesJobsQuery extends JerseyTest {
         .queryParam("finishedTimeBegin", String.valueOf(now))
         .queryParam("finishedTimeEnd", String.valueOf(40000))
         .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
-    assertEquals(Status.BAD_REQUEST, response.getClientResponseStatus());
+    assertResponseStatusCode(Status.BAD_REQUEST, response.getStatusInfo());
     assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
     JSONObject msg = response.getEntity(JSONObject.class);
     JSONObject exception = msg.getJSONObject("RemoteException");
@@ -565,7 +566,7 @@ public class TestHsWebServicesJobsQuery extends JerseyTest {
     ClientResponse response = r.path("ws").path("v1").path("history")
         .path("mapreduce").path("jobs").queryParam("finishedTimeBegin", "efsd")
         .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
-    assertEquals(Status.BAD_REQUEST, response.getClientResponseStatus());
+    assertResponseStatusCode(Status.BAD_REQUEST, response.getStatusInfo());
     assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
     JSONObject msg = response.getEntity(JSONObject.class);
     JSONObject exception = msg.getJSONObject("RemoteException");
@@ -591,7 +592,7 @@ public class TestHsWebServicesJobsQuery extends JerseyTest {
     ClientResponse response = r.path("ws").path("v1").path("history")
         .path("mapreduce").path("jobs").queryParam("finishedTimeEnd", "efsd")
         .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
-    assertEquals(Status.BAD_REQUEST, response.getClientResponseStatus());
+    assertResponseStatusCode(Status.BAD_REQUEST, response.getStatusInfo());
     assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
     JSONObject msg = response.getEntity(JSONObject.class);
     JSONObject exception = msg.getJSONObject("RemoteException");
@@ -639,7 +640,8 @@ public class TestHsWebServicesJobsQuery extends JerseyTest {
     assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
     JSONObject json = response.getEntity(JSONObject.class);
     assertEquals("incorrect number of elements", 1, json.length());
-    assertEquals("jobs is not null", JSONObject.NULL, json.get("jobs"));
+    assertEquals("jobs is not empty",
+        new JSONObject().toString(), json.get("jobs").toString());
   }
 
   @Test

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5d58858b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/test/java/org/apache/hadoop/mapreduce/v2/hs/webapp/TestHsWebServicesTasks.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/test/java/org/apache/hadoop/mapreduce/v2/hs/webapp/TestHsWebServicesTasks.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/test/java/org/apache/hadoop/mapreduce/v2/hs/webapp/TestHsWebServicesTasks.java
index 22fa46a..c05c151 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/test/java/org/apache/hadoop/mapreduce/v2/hs/webapp/TestHsWebServicesTasks.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/test/java/org/apache/hadoop/mapreduce/v2/hs/webapp/TestHsWebServicesTasks.java
@@ -18,6 +18,7 @@
 
 package org.apache.hadoop.mapreduce.v2.hs.webapp;
 
+import static org.apache.hadoop.yarn.webapp.WebServicesTestUtils.assertResponseStatusCode;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertTrue;
@@ -44,6 +45,7 @@ import org.apache.hadoop.mapreduce.v2.hs.HistoryContext;
 import org.apache.hadoop.mapreduce.v2.hs.MockHistoryContext;
 import org.apache.hadoop.mapreduce.v2.util.MRApps;
 import org.apache.hadoop.yarn.webapp.GenericExceptionHandler;
+import org.apache.hadoop.yarn.webapp.GuiceServletConfig;
 import org.apache.hadoop.yarn.webapp.WebApp;
 import org.apache.hadoop.yarn.webapp.WebServicesTestUtils;
 import org.codehaus.jettison.json.JSONArray;
@@ -57,8 +59,6 @@ import org.w3c.dom.NodeList;
 import org.xml.sax.InputSource;
 
 import com.google.inject.Guice;
-import com.google.inject.Injector;
-import com.google.inject.servlet.GuiceServletContextListener;
 import com.google.inject.servlet.ServletModule;
 import com.sun.jersey.api.client.ClientResponse;
 import com.sun.jersey.api.client.ClientResponse.Status;
@@ -82,10 +82,9 @@ public class TestHsWebServicesTasks extends JerseyTest {
   private static MockHistoryContext appContext;
   private static HsWebApp webApp;
 
-  private Injector injector = Guice.createInjector(new ServletModule() {
+  private static class WebServletModule extends ServletModule {
     @Override
     protected void configureServlets() {
-
       appContext = new MockHistoryContext(0, 1, 2, 1);
       webApp = mock(HsWebApp.class);
       when(webApp.name()).thenReturn("hsmockwebapp");
@@ -100,20 +99,19 @@ public class TestHsWebServicesTasks extends JerseyTest {
 
       serve("/*").with(GuiceContainer.class);
     }
-  });
-
-  public class GuiceServletConfig extends GuiceServletContextListener {
+  }
 
-    @Override
-    protected Injector getInjector() {
-      return injector;
-    }
+  static {
+    GuiceServletConfig.setInjector(
+        Guice.createInjector(new WebServletModule()));
   }
 
   @Before
   @Override
   public void setUp() throws Exception {
     super.setUp();
+    GuiceServletConfig.setInjector(
+        Guice.createInjector(new WebServletModule()));
   }
 
   public TestHsWebServicesTasks() {
@@ -266,7 +264,7 @@ public class TestHsWebServicesTasks extends JerseyTest {
         fail("should have thrown exception on invalid uri");
       } catch (UniformInterfaceException ue) {
         ClientResponse response = ue.getResponse();
-        assertEquals(Status.BAD_REQUEST, response.getClientResponseStatus());
+        assertResponseStatusCode(Status.BAD_REQUEST, response.getStatusInfo());
         assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
         JSONObject msg = response.getEntity(JSONObject.class);
         JSONObject exception = msg.getJSONObject("RemoteException");
@@ -361,7 +359,7 @@ public class TestHsWebServicesTasks extends JerseyTest {
         fail("should have thrown exception on invalid uri");
       } catch (UniformInterfaceException ue) {
         ClientResponse response = ue.getResponse();
-        assertEquals(Status.NOT_FOUND, response.getClientResponseStatus());
+        assertResponseStatusCode(Status.NOT_FOUND, response.getStatusInfo());
         assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
         JSONObject msg = response.getEntity(JSONObject.class);
         JSONObject exception = msg.getJSONObject("RemoteException");
@@ -395,7 +393,7 @@ public class TestHsWebServicesTasks extends JerseyTest {
         fail("should have thrown exception on invalid uri");
       } catch (UniformInterfaceException ue) {
         ClientResponse response = ue.getResponse();
-        assertEquals(Status.NOT_FOUND, response.getClientResponseStatus());
+        assertResponseStatusCode(Status.NOT_FOUND, response.getStatusInfo());
         assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
         JSONObject msg = response.getEntity(JSONObject.class);
         JSONObject exception = msg.getJSONObject("RemoteException");
@@ -427,7 +425,7 @@ public class TestHsWebServicesTasks extends JerseyTest {
         fail("should have thrown exception on invalid uri");
       } catch (UniformInterfaceException ue) {
         ClientResponse response = ue.getResponse();
-        assertEquals(Status.NOT_FOUND, response.getClientResponseStatus());
+        assertResponseStatusCode(Status.NOT_FOUND, response.getStatusInfo());
         assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
         JSONObject msg = response.getEntity(JSONObject.class);
         JSONObject exception = msg.getJSONObject("RemoteException");
@@ -461,7 +459,7 @@ public class TestHsWebServicesTasks extends JerseyTest {
         fail("should have thrown exception on invalid uri");
       } catch (UniformInterfaceException ue) {
         ClientResponse response = ue.getResponse();
-        assertEquals(Status.NOT_FOUND, response.getClientResponseStatus());
+        assertResponseStatusCode(Status.NOT_FOUND, response.getStatusInfo());
         assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
         JSONObject msg = response.getEntity(JSONObject.class);
         JSONObject exception = msg.getJSONObject("RemoteException");
@@ -495,7 +493,7 @@ public class TestHsWebServicesTasks extends JerseyTest {
         fail("should have thrown exception on invalid uri");
       } catch (UniformInterfaceException ue) {
         ClientResponse response = ue.getResponse();
-        assertEquals(Status.NOT_FOUND, response.getClientResponseStatus());
+        assertResponseStatusCode(Status.NOT_FOUND, response.getStatusInfo());
         assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
         JSONObject msg = response.getEntity(JSONObject.class);
         JSONObject exception = msg.getJSONObject("RemoteException");

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5d58858b/hadoop-project/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-project/pom.xml b/hadoop-project/pom.xml
index 7558b04..8810eff 100644
--- a/hadoop-project/pom.xml
+++ b/hadoop-project/pom.xml
@@ -65,7 +65,7 @@
     <avro.version>1.7.4</avro.version>
 
     <!-- jersey version -->
-    <jersey.version>1.9</jersey.version>
+    <jersey.version>1.19</jersey.version>
 
     <!-- jackson versions -->
     <jackson.version>1.9.13</jackson.version>
@@ -548,6 +548,11 @@
       </dependency>
       <dependency>
         <groupId>com.sun.jersey</groupId>
+        <artifactId>jersey-servlet</artifactId>
+        <version>${jersey.version}</version>
+      </dependency>
+      <dependency>
+        <groupId>com.sun.jersey</groupId>
         <artifactId>jersey-json</artifactId>
         <version>${jersey.version}</version>
         <exclusions>
@@ -946,7 +951,17 @@
       <dependency>
         <groupId>org.glassfish.grizzly</groupId>
         <artifactId>grizzly-http-servlet</artifactId>
-        <version>2.1.2</version>
+        <version>2.2.21</version>
+      </dependency>
+      <dependency>
+        <groupId>org.glassfish.grizzly</groupId>
+        <artifactId>grizzly-http</artifactId>
+        <version>2.2.21</version>
+      </dependency>
+      <dependency>
+        <groupId>org.glassfish.grizzly</groupId>
+        <artifactId>grizzly-http-server</artifactId>
+        <version>2.2.21</version>
       </dependency>
 
       <dependency>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5d58858b/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 d28c77c..411d0bc 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
@@ -342,7 +342,8 @@ public class LogsCLI extends Configured implements Tool {
           webResource.path("ws").path("v1").path("node").path("containers")
               .path(containerIdStr).accept(MediaType.APPLICATION_XML)
               .get(ClientResponse.class);
-      if (response.getClientResponseStatus().equals(ClientResponse.Status.OK)) {
+      if (response.getStatusInfo().getStatusCode() ==
+          ClientResponse.Status.OK.getStatusCode()) {
         try {
           String xml = response.getEntity(String.class);
           DocumentBuilderFactory dbf = DocumentBuilderFactory.newInstance();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5d58858b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/pom.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/pom.xml
index 17fc6e2..1519553 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/pom.xml
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/pom.xml
@@ -42,6 +42,7 @@
       <scope>provided</scope>
     </dependency>
 
+
     <!-- 'mvn dependency:analyze' fails to detect use of this dependency -->
     <dependency>
       <groupId>org.apache.hadoop</groupId>
@@ -49,6 +50,11 @@
     </dependency>
 
     <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-auth</artifactId>
+    </dependency>
+
+    <dependency>
       <groupId>javax.xml.bind</groupId>
       <artifactId>jaxb-api</artifactId>
     </dependency>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5d58858b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/impl/TimelineWriter.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/impl/TimelineWriter.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/impl/TimelineWriter.java
index 9590f4a..8fd0990 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/impl/TimelineWriter.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/impl/TimelineWriter.java
@@ -126,7 +126,8 @@ public abstract class TimelineWriter implements Flushable {
       throw (IOException)new InterruptedIOException().initCause(ie);
     }
     if (resp == null ||
-        resp.getClientResponseStatus() != ClientResponse.Status.OK) {
+        resp.getStatusInfo().getStatusCode()
+            != ClientResponse.Status.OK.getStatusCode()) {
       String msg =
           "Failed to get the response from the timeline server.";
       LOG.error(msg);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5d58858b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestTimelineClient.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestTimelineClient.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestTimelineClient.java
index 41b788d..ddf6c32 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestTimelineClient.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestTimelineClient.java
@@ -330,7 +330,7 @@ public class TestTimelineClient {
     }
     doReturn(response).when(spyTimelineWriter)
         .doPostingObject(any(TimelineEntities.class), any(String.class));
-    when(response.getClientResponseStatus()).thenReturn(status);
+    when(response.getStatusInfo()).thenReturn(status);
     TimelinePutResponse.TimelinePutError error =
         new TimelinePutResponse.TimelinePutError();
     error.setEntityId("test entity id");
@@ -356,7 +356,7 @@ public class TestTimelineClient {
     }
     doReturn(response).when(spyTimelineWriter)
         .doPostingObject(any(TimelineDomain.class), any(String.class));
-    when(response.getClientResponseStatus()).thenReturn(status);
+    when(response.getStatusInfo()).thenReturn(status);
     return response;
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5d58858b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestTimelineClientForATS1_5.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestTimelineClientForATS1_5.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestTimelineClientForATS1_5.java
index 37eadbf..d3826e1 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestTimelineClientForATS1_5.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestTimelineClientForATS1_5.java
@@ -208,8 +208,8 @@ public class TestTimelineClientForATS1_5 {
             new FileSystemTimelineWriter(conf, authUgi, client, resURI) {
               public ClientResponse doPostingObject(Object object, String path) {
                 ClientResponse response = mock(ClientResponse.class);
-                when(response.getClientResponseStatus()).thenReturn(
-                  ClientResponse.Status.OK);
+                when(response.getStatusInfo()).thenReturn(
+                    ClientResponse.Status.OK);
                 return response;
               }
             };

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5d58858b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/webapp/GuiceServletConfig.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/webapp/GuiceServletConfig.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/webapp/GuiceServletConfig.java
new file mode 100644
index 0000000..5ce78c0
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/webapp/GuiceServletConfig.java
@@ -0,0 +1,43 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.yarn.webapp;
+
+import com.google.inject.Injector;
+import com.google.inject.servlet.GuiceServletContextListener;
+
+/**
+ * GuiceServletConfig is a wrapper class to have a static Injector instance
+ * instead of having the instance inside test classes. This allow us to use
+ * Jersey test framework after 1.13.
+ * Please check test cases to know how to use this class:
+ * e.g. TestRMWithCSRFFilter.java
+ */
+public class GuiceServletConfig extends GuiceServletContextListener {
+
+  private static Injector internalInjector = null;
+
+  @Override
+  protected Injector getInjector() {
+    return internalInjector;
+  }
+
+  public static Injector setInjector(Injector in) {
+    internalInjector = in;
+    return internalInjector;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5d58858b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/webapp/WebServicesTestUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/webapp/WebServicesTestUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/webapp/WebServicesTestUtils.java
index 0454ce6..c421eb9 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/webapp/WebServicesTestUtils.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/webapp/WebServicesTestUtils.java
@@ -18,10 +18,12 @@
 
 package org.apache.hadoop.yarn.webapp;
 
+import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 
 import java.util.ArrayList;
 import java.util.List;
+import javax.ws.rs.core.Response.StatusType;
 
 import org.w3c.dom.Attr;
 import org.w3c.dom.Element;
@@ -119,4 +121,13 @@ public class WebServicesTestUtils {
         got.equals(expected));
   }
 
+  public static void assertResponseStatusCode(StatusType expected,
+      StatusType actual) {
+    assertResponseStatusCode(null, expected, actual);
+  }
+
+  public static void assertResponseStatusCode(String errmsg,
+      StatusType expected, StatusType actual) {
+    assertEquals(errmsg, expected.getStatusCode(), actual.getStatusCode());
+  }
 }


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


[05/17] hadoop git commit: HADOOP-13288. Guard null stats key in FileSystemStorageStatistics (Mingliang Liu via Colin P. McCabe)

Posted by ae...@apache.org.
HADOOP-13288. Guard null stats key in FileSystemStorageStatistics (Mingliang Liu 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/8c1f81d4
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/8c1f81d4
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/8c1f81d4

Branch: refs/heads/HDFS-1312
Commit: 8c1f81d4bf424bdc421cf4952b230344e39a7b68
Parents: aa1b583
Author: Colin Patrick Mccabe <cm...@cloudera.com>
Authored: Mon Jun 20 14:25:07 2016 -0700
Committer: Colin Patrick Mccabe <cm...@cloudera.com>
Committed: Mon Jun 20 14:25:07 2016 -0700

----------------------------------------------------------------------
 .../org/apache/hadoop/fs/FileSystemStorageStatistics.java    | 8 ++++++++
 1 file changed, 8 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/8c1f81d4/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileSystemStorageStatistics.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileSystemStorageStatistics.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileSystemStorageStatistics.java
index f179cce..7c08863 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileSystemStorageStatistics.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileSystemStorageStatistics.java
@@ -20,6 +20,7 @@ package org.apache.hadoop.fs;
 import java.util.Iterator;
 import java.util.NoSuchElementException;
 
+import com.google.common.base.Preconditions;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.fs.FileSystem.Statistics.StatisticsData;
@@ -81,6 +82,9 @@ public class FileSystemStorageStatistics extends StorageStatistics {
   }
 
   private static Long fetch(StatisticsData data, String key) {
+    Preconditions.checkArgument(key != null,
+        "The stat key of FileSystemStorageStatistics should not be null!");
+
     switch (key) {
     case "bytesRead":
       return data.getBytesRead();
@@ -107,6 +111,10 @@ public class FileSystemStorageStatistics extends StorageStatistics {
 
   FileSystemStorageStatistics(String name, FileSystem.Statistics stats) {
     super(name);
+    Preconditions.checkArgument(stats != null,
+        "FileSystem.Statistics can not be null");
+    Preconditions.checkArgument(stats.getData() != null,
+        "FileSystem.Statistics can not have null data");
     this.stats = stats;
   }
 


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


[02/17] hadoop git commit: HDFS-10328. Add per-cache-pool default replication num configuration (xupeng via cmccabe)

Posted by ae...@apache.org.
HDFS-10328. Add per-cache-pool default replication num configuration (xupeng via cmccabe)


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

Branch: refs/heads/HDFS-1312
Commit: 5f6b4157a40e974ccc6a56c39dbd35c54f393fbd
Parents: 5107a96
Author: Colin Patrick Mccabe <cm...@cloudera.com>
Authored: Fri Jun 17 12:44:36 2016 -0700
Committer: Colin Patrick Mccabe <cm...@cloudera.com>
Committed: Mon Jun 20 10:42:50 2016 -0700

----------------------------------------------------------------------
 .../hadoop/hdfs/protocol/CachePoolInfo.java     | 25 ++++++++++
 .../hadoop/hdfs/protocolPB/PBHelperClient.java  |  7 +++
 .../src/main/proto/ClientNamenodeProtocol.proto |  1 +
 .../hdfs/server/namenode/CacheManager.java      | 14 +++++-
 .../hadoop/hdfs/server/namenode/CachePool.java  | 28 +++++++++--
 .../server/namenode/FSImageSerialization.java   | 16 ++++--
 .../apache/hadoop/hdfs/tools/CacheAdmin.java    | 34 +++++++++++--
 .../src/test/resources/testCacheAdminConf.xml   | 51 ++++++++++++++++----
 8 files changed, 157 insertions(+), 19 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/5f6b4157/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/CachePoolInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/CachePoolInfo.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/CachePoolInfo.java
index c8a70ac..daa77be 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/CachePoolInfo.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/CachePoolInfo.java
@@ -54,6 +54,8 @@ public class CachePoolInfo {
   public static final long LIMIT_UNLIMITED = Long.MAX_VALUE;
   public static final long DEFAULT_LIMIT = LIMIT_UNLIMITED;
 
+  public static final short DEFAULT_REPLICATION_NUM = 1;
+
   final String poolName;
 
   @Nullable
@@ -69,6 +71,9 @@ public class CachePoolInfo {
   Long limit;
 
   @Nullable
+  private Short defaultReplication;
+
+  @Nullable
   Long maxRelativeExpiryMs;
 
   public CachePoolInfo(String poolName) {
@@ -135,6 +140,18 @@ public class CachePoolInfo {
   }
 
   /**
+   * @return The default replication num for CacheDirective in this pool
+     */
+  public Short getDefaultReplication() {
+    return defaultReplication;
+  }
+
+  public CachePoolInfo setDefaultReplication(Short repl) {
+    this.defaultReplication = repl;
+    return this;
+  }
+
+  /**
    * @return The maximum relative expiration of directives of this pool in
    *         milliseconds
    */
@@ -161,6 +178,7 @@ public class CachePoolInfo {
         + ", mode:"
         + ((mode == null) ? "null" : String.format("0%03o", mode.toShort()))
         + ", limit:" + limit
+        + ", defaultReplication:" + defaultReplication
         + ", maxRelativeExpiryMs:" + maxRelativeExpiryMs + "}";
   }
 
@@ -178,6 +196,7 @@ public class CachePoolInfo {
         append(groupName, other.groupName).
         append(mode, other.mode).
         append(limit, other.limit).
+        append(defaultReplication, other.defaultReplication).
         append(maxRelativeExpiryMs, other.maxRelativeExpiryMs).
         isEquals();
   }
@@ -190,6 +209,7 @@ public class CachePoolInfo {
         append(groupName).
         append(mode).
         append(limit).
+        append(defaultReplication).
         append(maxRelativeExpiryMs).
         hashCode();
   }
@@ -201,6 +221,11 @@ public class CachePoolInfo {
     if ((info.getLimit() != null) && (info.getLimit() < 0)) {
       throw new InvalidRequestException("Limit is negative.");
     }
+    if ((info.getDefaultReplication() != null)
+            && (info.getDefaultReplication() < 0)) {
+      throw new InvalidRequestException("Default Replication is negative");
+    }
+
     if (info.getMaxRelativeExpiryMs() != null) {
       long maxRelativeExpiryMs = info.getMaxRelativeExpiryMs();
       if (maxRelativeExpiryMs < 0l) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5f6b4157/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelperClient.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelperClient.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelperClient.java
index d5bb1e7..3c5f583 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelperClient.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelperClient.java
@@ -1203,6 +1203,10 @@ public class PBHelperClient {
     if (proto.hasLimit())  {
       info.setLimit(proto.getLimit());
     }
+    if (proto.hasDefaultReplication()) {
+      info.setDefaultReplication(Shorts.checkedCast(
+          proto.getDefaultReplication()));
+    }
     if (proto.hasMaxRelativeExpiry()) {
       info.setMaxRelativeExpiryMs(proto.getMaxRelativeExpiry());
     }
@@ -1234,6 +1238,9 @@ public class PBHelperClient {
     if (info.getLimit() != null) {
       builder.setLimit(info.getLimit());
     }
+    if (info.getDefaultReplication() != null) {
+      builder.setDefaultReplication(info.getDefaultReplication());
+    }
     if (info.getMaxRelativeExpiryMs() != null) {
       builder.setMaxRelativeExpiry(info.getMaxRelativeExpiryMs());
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5f6b4157/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/ClientNamenodeProtocol.proto
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/ClientNamenodeProtocol.proto b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/ClientNamenodeProtocol.proto
index 7acb394..aac48a4 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/ClientNamenodeProtocol.proto
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/ClientNamenodeProtocol.proto
@@ -534,6 +534,7 @@ message CachePoolInfoProto {
   optional int32 mode = 4;
   optional int64 limit = 5;
   optional int64 maxRelativeExpiry = 6;
+  optional uint32 defaultReplication = 7 [default=1];
 }
 
 message CachePoolStatsProto {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5f6b4157/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/CacheManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/CacheManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/CacheManager.java
index b1f936b..cd57c56 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/CacheManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/CacheManager.java
@@ -523,7 +523,8 @@ public final class CacheManager {
       CachePool pool = getCachePool(validatePoolName(info));
       checkWritePermission(pc, pool);
       String path = validatePath(info);
-      short replication = validateReplication(info, (short)1);
+      short replication = validateReplication(
+              info, pool.getDefaultReplication());
       long expiryTime = validateExpiryTime(info, pool.getMaxRelativeExpiryMs());
       // Do quota validation if required
       if (!flags.contains(CacheFlag.FORCE)) {
@@ -826,6 +827,13 @@ public final class CacheManager {
         // New limit changes stats, need to set needs refresh
         setNeedsRescan();
       }
+      if (info.getDefaultReplication() != null) {
+        final short defaultReplication = info.getDefaultReplication();
+        pool.setDefaultReplication(defaultReplication);
+        bld.append(prefix).append("set default replication to "
+            + defaultReplication);
+        prefix = "; ";
+      }
       if (info.getMaxRelativeExpiryMs() != null) {
         final Long maxRelativeExpiry = info.getMaxRelativeExpiryMs();
         pool.setMaxRelativeExpiryMs(maxRelativeExpiry);
@@ -1083,6 +1091,10 @@ public final class CacheManager {
       if (p.hasMode())
         info.setMode(new FsPermission((short) p.getMode()));
 
+      if (p.hasDefaultReplication()) {
+        info.setDefaultReplication((short) p.getDefaultReplication());
+      }
+
       if (p.hasLimit())
         info.setLimit(p.getLimit());
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5f6b4157/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/CachePool.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/CachePool.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/CachePool.java
index 585124f..a2613d9 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/CachePool.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/CachePool.java
@@ -73,6 +73,11 @@ public final class CachePool {
   private long limit;
 
   /**
+   * Default replication num for CacheDirective in this pool.
+   */
+  private short defaultReplication;
+
+  /**
    * Maximum duration that a CacheDirective in this pool remains valid,
    * in milliseconds.
    */
@@ -123,11 +128,15 @@ public final class CachePool {
         FsPermission.getCachePoolDefault() : info.getMode();
     long limit = info.getLimit() == null ?
         CachePoolInfo.DEFAULT_LIMIT : info.getLimit();
+    short defaultReplication = info.getDefaultReplication() == null ?
+        CachePoolInfo.DEFAULT_REPLICATION_NUM :
+        info.getDefaultReplication();
     long maxRelativeExpiry = info.getMaxRelativeExpiryMs() == null ?
         CachePoolInfo.DEFAULT_MAX_RELATIVE_EXPIRY :
         info.getMaxRelativeExpiryMs();
     return new CachePool(info.getPoolName(),
-        ownerName, groupName, mode, limit, maxRelativeExpiry);
+        ownerName, groupName, mode, limit,
+        defaultReplication, maxRelativeExpiry);
   }
 
   /**
@@ -137,11 +146,13 @@ public final class CachePool {
   static CachePool createFromInfo(CachePoolInfo info) {
     return new CachePool(info.getPoolName(),
         info.getOwnerName(), info.getGroupName(),
-        info.getMode(), info.getLimit(), info.getMaxRelativeExpiryMs());
+        info.getMode(), info.getLimit(),
+        info.getDefaultReplication(), info.getMaxRelativeExpiryMs());
   }
 
   CachePool(String poolName, String ownerName, String groupName,
-      FsPermission mode, long limit, long maxRelativeExpiry) {
+      FsPermission mode, long limit,
+      short defaultReplication, long maxRelativeExpiry) {
     Preconditions.checkNotNull(poolName);
     Preconditions.checkNotNull(ownerName);
     Preconditions.checkNotNull(groupName);
@@ -151,6 +162,7 @@ public final class CachePool {
     this.groupName = groupName;
     this.mode = new FsPermission(mode);
     this.limit = limit;
+    this.defaultReplication = defaultReplication;
     this.maxRelativeExpiryMs = maxRelativeExpiry;
   }
 
@@ -194,6 +206,14 @@ public final class CachePool {
     return this;
   }
 
+  public short getDefaultReplication() {
+    return defaultReplication;
+  }
+
+  public void setDefaultReplication(short replication) {
+    this.defaultReplication = replication;
+  }
+
   public long getMaxRelativeExpiryMs() {
     return maxRelativeExpiryMs;
   }
@@ -221,6 +241,7 @@ public final class CachePool {
         setGroupName(groupName).
         setMode(new FsPermission(mode)).
         setLimit(limit).
+        setDefaultReplication(defaultReplication).
         setMaxRelativeExpiryMs(maxRelativeExpiryMs);
   }
 
@@ -314,6 +335,7 @@ public final class CachePool {
         append(", groupName:").append(groupName).
         append(", mode:").append(mode).
         append(", limit:").append(limit).
+        append(", defaultReplication").append(defaultReplication).
         append(", maxRelativeExpiryMs:").append(maxRelativeExpiryMs).
         append(" }").toString();
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5f6b4157/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageSerialization.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageSerialization.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageSerialization.java
index 419a704..06ac6a7 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageSerialization.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageSerialization.java
@@ -619,20 +619,24 @@ public class FSImageSerialization {
     final Long limit = info.getLimit();
     final FsPermission mode = info.getMode();
     final Long maxRelativeExpiry = info.getMaxRelativeExpiryMs();
+    final Short defaultReplication = info.getDefaultReplication();
 
-    boolean hasOwner, hasGroup, hasMode, hasLimit, hasMaxRelativeExpiry;
+    boolean hasOwner, hasGroup, hasMode, hasLimit,
+            hasMaxRelativeExpiry, hasDefaultReplication;
     hasOwner = ownerName != null;
     hasGroup = groupName != null;
     hasMode = mode != null;
     hasLimit = limit != null;
     hasMaxRelativeExpiry = maxRelativeExpiry != null;
+    hasDefaultReplication = defaultReplication != null;
 
     int flags =
         (hasOwner ? 0x1 : 0) |
         (hasGroup ? 0x2 : 0) |
         (hasMode  ? 0x4 : 0) |
         (hasLimit ? 0x8 : 0) |
-        (hasMaxRelativeExpiry ? 0x10 : 0);
+        (hasMaxRelativeExpiry ? 0x10 : 0) |
+        (hasDefaultReplication ? 0x20 : 0);
 
     writeInt(flags, out);
 
@@ -651,6 +655,9 @@ public class FSImageSerialization {
     if (hasMaxRelativeExpiry) {
       writeLong(maxRelativeExpiry, out);
     }
+    if (hasDefaultReplication) {
+      writeShort(defaultReplication, out);
+    }
   }
 
   public static CachePoolInfo readCachePoolInfo(DataInput in)
@@ -673,7 +680,10 @@ public class FSImageSerialization {
     if ((flags & 0x10) != 0) {
       info.setMaxRelativeExpiryMs(readLong(in));
     }
-    if ((flags & ~0x1F) != 0) {
+    if ((flags & 0x20) != 0) {
+      info.setDefaultReplication(readShort(in));
+    }
+    if ((flags & ~0x2F) != 0) {
       throw new IOException("Unknown flag in CachePoolInfo: " + flags);
     }
     return info;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5f6b4157/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/CacheAdmin.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/CacheAdmin.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/CacheAdmin.java
index 7912c3a..270a662 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/CacheAdmin.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/CacheAdmin.java
@@ -555,7 +555,7 @@ public class CacheAdmin extends Configured implements Tool {
     public String getShortUsage() {
       return "[" + NAME + " <name> [-owner <owner>] " +
           "[-group <group>] [-mode <mode>] [-limit <limit>] " +
-          "[-maxTtl <maxTtl>]" +
+          "[-defaultReplication <defaultReplication>] [-maxTtl <maxTtl>]" +
           "]\n";
     }
 
@@ -575,6 +575,9 @@ public class CacheAdmin extends Configured implements Tool {
       listing.addRow("<limit>", "The maximum number of bytes that can be " +
           "cached by directives in this pool, in aggregate. By default, " +
           "no limit is set.");
+      listing.addRow("<defaultReplication>", "The default replication " +
+          "number for cache directive in the pool. " +
+          "If not set, the replication is set to 1");
       listing.addRow("<maxTtl>", "The maximum allowed time-to-live for " +
           "directives being added to the pool. This can be specified in " +
           "seconds, minutes, hours, and days, e.g. 120s, 30m, 4h, 2d. " +
@@ -613,6 +616,12 @@ public class CacheAdmin extends Configured implements Tool {
       if (limit != null) {
         info.setLimit(limit);
       }
+      String replicationString = StringUtils.
+              popOptionWithArgument("-defaultReplication", args);
+      if (replicationString != null) {
+        short defaultReplication = Short.parseShort(replicationString);
+        info.setDefaultReplication(defaultReplication);
+      }
       String maxTtlString = StringUtils.popOptionWithArgument("-maxTtl", args);
       try {
         Long maxTtl = AdminHelper.parseTtlString(maxTtlString);
@@ -654,7 +663,7 @@ public class CacheAdmin extends Configured implements Tool {
     public String getShortUsage() {
       return "[" + getName() + " <name> [-owner <owner>] " +
           "[-group <group>] [-mode <mode>] [-limit <limit>] " +
-          "[-maxTtl <maxTtl>]]\n";
+          "[-defaultReplication <defaultReplication>] [-maxTtl <maxTtl>]]\n";
     }
 
     @Override
@@ -667,6 +676,8 @@ public class CacheAdmin extends Configured implements Tool {
       listing.addRow("<mode>", "Unix-style permissions of the pool in octal.");
       listing.addRow("<limit>", "Maximum number of bytes that can be cached " +
           "by this pool.");
+      listing.addRow("<defaultReplication>", "Default replication num for " +
+          "directives in this pool");
       listing.addRow("<maxTtl>", "The maximum allowed time-to-live for " +
           "directives being added to the pool.");
 
@@ -686,6 +697,12 @@ public class CacheAdmin extends Configured implements Tool {
           null : Integer.parseInt(modeString, 8);
       String limitString = StringUtils.popOptionWithArgument("-limit", args);
       Long limit = AdminHelper.parseLimitString(limitString);
+      String replicationString =
+              StringUtils.popOptionWithArgument("-defaultReplication", args);
+      Short defaultReplication = null;
+      if (replicationString != null) {
+        defaultReplication = Short.parseShort(replicationString);
+      }
       String maxTtlString = StringUtils.popOptionWithArgument("-maxTtl", args);
       Long maxTtl;
       try {
@@ -725,6 +742,10 @@ public class CacheAdmin extends Configured implements Tool {
         info.setLimit(limit);
         changed = true;
       }
+      if (defaultReplication != null) {
+        info.setDefaultReplication(defaultReplication);
+        changed = true;
+      }
       if (maxTtl != null) {
         info.setMaxRelativeExpiryMs(maxTtl);
         changed = true;
@@ -759,6 +780,10 @@ public class CacheAdmin extends Configured implements Tool {
         System.out.print(prefix + "limit " + limit);
         prefix = " and ";
       }
+      if (defaultReplication != null) {
+        System.out.println(prefix + "replication " + defaultReplication);
+        prefix = " replication ";
+      }
       if (maxTtl != null) {
         System.out.print(prefix + "max time-to-live " + maxTtlString);
       }
@@ -854,7 +879,8 @@ public class CacheAdmin extends Configured implements Tool {
           addField("GROUP", Justification.LEFT).
           addField("MODE", Justification.LEFT).
           addField("LIMIT", Justification.RIGHT).
-          addField("MAXTTL", Justification.RIGHT);
+          addField("MAXTTL", Justification.RIGHT).
+          addField("DEFAULT_REPLICATION", Justification.RIGHT);
       if (printStats) {
         builder.
             addField("BYTES_NEEDED", Justification.RIGHT).
@@ -895,6 +921,8 @@ public class CacheAdmin extends Configured implements Tool {
               }
             }
             row.add(maxTtlString);
+            row.add("" + info.getDefaultReplication());
+
             if (printStats) {
               CachePoolStats stats = entry.getStats();
               row.add(Long.toString(stats.getBytesNeeded()));

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5f6b4157/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testCacheAdminConf.xml
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testCacheAdminConf.xml b/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testCacheAdminConf.xml
index 058eec5..479deb5 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testCacheAdminConf.xml
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testCacheAdminConf.xml
@@ -80,8 +80,8 @@
     <test> <!--Tested -->
       <description>Testing modifying a cache pool</description>
       <test-commands>
-        <cache-admin-command>-addPool poolparty -owner alice -group alicegroup -mode 0000 -limit 50</cache-admin-command>
-        <cache-admin-command>-modifyPool poolparty -owner bob -group bobgroup -mode 0777 -limit 51</cache-admin-command>
+        <cache-admin-command>-addPool poolparty -owner alice -group alicegroup -mode 0000 -limit 50 -defaultReplication 3</cache-admin-command>
+        <cache-admin-command>-modifyPool poolparty -owner bob -group bobgroup -mode 0777 -limit 51 -defaultReplication 2</cache-admin-command>
         <cache-admin-command>-listPools</cache-admin-command>
       </test-commands>
       <cleanup-commands>
@@ -90,11 +90,12 @@
       <comparators>
         <comparator>
           <type>SubstringComparator</type>
-          <expected-output>poolparty  bob    bobgroup  rwxrwxrwx      51</expected-output>
+          <expected-output>poolparty  bob    bobgroup  rwxrwxrwx      51   never                    2</expected-output>
         </comparator>
       </comparators>
     </test>
 
+
     <test> <!--Tested -->
       <description>Testing deleting a cache pool</description>
       <test-commands>
@@ -114,7 +115,7 @@
     <test> <!--Tested -->
       <description>Testing listing all cache pools</description>
       <test-commands>
-        <cache-admin-command>-addPool foo -owner bob -group bob -mode 0664</cache-admin-command>
+        <cache-admin-command>-addPool foo -owner bob -group bob -mode 0664 -defaultReplication 2</cache-admin-command>
         <cache-admin-command>-addPool bar -owner alice -group alicegroup -mode 0755</cache-admin-command>
         <cache-admin-command>-listPools</cache-admin-command>
       </test-commands>
@@ -129,11 +130,11 @@
         </comparator>
         <comparator>
           <type>SubstringComparator</type>
-          <expected-output>bar   alice  alicegroup  rwxr-xr-x   unlimited</expected-output>
+          <expected-output>bar   alice  alicegroup  rwxr-xr-x   unlimited   never                    1</expected-output>
         </comparator>
         <comparator>
           <type>SubstringComparator</type>
-          <expected-output>foo   bob    bob         rw-rw-r--   unlimited</expected-output>
+          <expected-output>foo   bob    bob         rw-rw-r--   unlimited   never                    2</expected-output>
         </comparator>
       </comparators>
     </test>
@@ -156,7 +157,7 @@
         </comparator>
         <comparator>
           <type>SubstringComparator</type>
-          <expected-output>foo   bob    bob    rw-rw-r--   unlimited</expected-output>
+          <expected-output>foo   bob    bob    rw-rw-r--   unlimited   never                    1</expected-output>
         </comparator>
       </comparators>
     </test>
@@ -417,11 +418,11 @@
         </comparator>
         <comparator>
           <type>SubstringComparator</type>
-          <expected-output>bar   alice  alicegroup  rwxr-xr-x   unlimited   never             0             0                0             0             0</expected-output>
+          <expected-output>bar   alice  alicegroup  rwxr-xr-x   unlimited   never                    1             0             0                0             0             0</expected-output>
         </comparator>
         <comparator>
           <type>SubstringComparator</type>
-          <expected-output>foo   bob    bob         rw-rw-r--   unlimited   never             0             0                0             0             0</expected-output>
+          <expected-output>foo   bob    bob         rw-rw-r--   unlimited   never                    1             0             0                0             0             0</expected-output>
         </comparator>
       </comparators>
     </test>
@@ -543,5 +544,37 @@
         </comparator>
       </comparators>
     </test>
+
+    <test> <!--Tested -->
+      <description>Testing overriding cache pool replication</description>
+      <test-commands>
+        <cache-admin-command>-addPool pool1 -defaultReplication 2</cache-admin-command>
+        <cache-admin-command>-addPool pool2 -defaultReplication 3</cache-admin-command>
+        <cache-admin-command>-addDirective -path /foo -pool pool1 -ttl 2d</cache-admin-command>
+        <cache-admin-command>-addDirective -path /bar -pool pool2 -ttl 24h</cache-admin-command>
+        <cache-admin-command>-modifyDirective -id 32 -replication 3</cache-admin-command>
+        <cache-admin-command>-modifyDirective -id 34 -pool pool1</cache-admin-command>
+        <cache-admin-command>-addDirective -path /baz -replication 3 -pool pool1 -ttl 60m</cache-admin-command>
+        <cache-admin-command>-listDirectives -pool pool1</cache-admin-command>
+      </test-commands>
+      <cleanup-commands>
+        <cache-admin-command>-removePool pool1</cache-admin-command>
+        <cache-admin-command>-removePool pool2</cache-admin-command>
+      </cleanup-commands>
+      <comparators>
+        <comparator>
+          <type>SubstringComparator</type>
+          <expected-output>Found 2 entries</expected-output>
+        </comparator>
+        <comparator>
+          <type>SubstringComparator</type>
+          <expected-output>32 pool1      3</expected-output>
+        </comparator>
+        <comparator>
+          <type>SubstringComparator</type>
+          <expected-output>34 pool1      3</expected-output>
+        </comparator>
+      </comparators>
+    </test>
   </tests>
 </configuration>


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


[16/17] hadoop git commit: MAPREDUCE-6197. Cache MapOutputLocations in ShuffleHandler. Contributed by Junping Du

Posted by ae...@apache.org.
MAPREDUCE-6197. Cache MapOutputLocations in ShuffleHandler. Contributed by Junping Du


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

Branch: refs/heads/HDFS-1312
Commit: d8107fcd1c93c202925f2946d0cd4072fe0aef1e
Parents: b2c596c
Author: Jian He <ji...@apache.org>
Authored: Tue Jun 21 14:25:58 2016 -0700
Committer: Jian He <ji...@apache.org>
Committed: Tue Jun 21 14:25:58 2016 -0700

----------------------------------------------------------------------
 .../apache/hadoop/mapred/ShuffleHandler.java    | 193 +++++++++++++++----
 .../hadoop/mapred/TestShuffleHandler.java       |  16 +-
 2 files changed, 165 insertions(+), 44 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8107fcd/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 8cbae81..ed197f2 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
@@ -46,6 +46,7 @@ import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ExecutionException;
 import java.util.concurrent.ThreadFactory;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicInteger;
@@ -131,6 +132,12 @@ import org.mortbay.jetty.HttpHeaders;
 
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Charsets;
+import com.google.common.cache.CacheBuilder;
+import com.google.common.cache.CacheLoader;
+import com.google.common.cache.LoadingCache;
+import com.google.common.cache.RemovalListener;
+import com.google.common.cache.RemovalNotification;
+import com.google.common.cache.Weigher;
 import com.google.common.util.concurrent.ThreadFactoryBuilder;
 import com.google.protobuf.ByteString;
 
@@ -156,6 +163,9 @@ public class ShuffleHandler extends AuxiliaryService {
   protected static final Version CURRENT_VERSION_INFO = 
       Version.newInstance(1, 0);
 
+  private static final String DATA_FILE_NAME = "file.out";
+  private static final String INDEX_FILE_NAME = "file.out.index";
+
   private int port;
   private ChannelFactory selector;
   private final ChannelGroup accepted = new DefaultChannelGroup();
@@ -294,12 +304,12 @@ public class ShuffleHandler extends AuxiliaryService {
     private ChannelHandlerContext ctx;
     private String user;
     private Map<String, Shuffle.MapOutputInfo> infoMap;
-    private String outputBasePathStr;
+    private String jobId;
 
     public ReduceContext(List<String> mapIds, int rId,
                          ChannelHandlerContext context, String usr,
                          Map<String, Shuffle.MapOutputInfo> mapOutputInfoMap,
-                         String outputBasePath) {
+                         String jobId) {
 
       this.mapIds = mapIds;
       this.reduceId = rId;
@@ -319,7 +329,7 @@ public class ShuffleHandler extends AuxiliaryService {
       this.ctx = context;
       this.user = usr;
       this.infoMap = mapOutputInfoMap;
-      this.outputBasePathStr = outputBasePath;
+      this.jobId = jobId;
     }
 
     public int getReduceId() {
@@ -338,8 +348,8 @@ public class ShuffleHandler extends AuxiliaryService {
       return infoMap;
     }
 
-    public String getOutputBasePathStr() {
-      return outputBasePathStr;
+    public String getJobId() {
+      return jobId;
     }
 
     public List<String> getMapIds() {
@@ -780,18 +790,63 @@ public class ShuffleHandler extends AuxiliaryService {
 
   class Shuffle extends SimpleChannelUpstreamHandler {
 
+    private static final int MAX_WEIGHT = 10 * 1024 * 1024;
+    private static final int EXPIRE_AFTER_ACCESS_MINUTES = 5;
+    private static final int ALLOWED_CONCURRENCY = 16;
     private final Configuration conf;
     private final IndexCache indexCache;
     private final LocalDirAllocator lDirAlloc =
       new LocalDirAllocator(YarnConfiguration.NM_LOCAL_DIRS);
     private int port;
+    private final LoadingCache<AttemptPathIdentifier, AttemptPathInfo> pathCache =
+      CacheBuilder.newBuilder().expireAfterAccess(EXPIRE_AFTER_ACCESS_MINUTES,
+      TimeUnit.MINUTES).softValues().concurrencyLevel(ALLOWED_CONCURRENCY).
+      removalListener(
+          new RemovalListener<AttemptPathIdentifier, AttemptPathInfo>() {
+            @Override
+            public void onRemoval(RemovalNotification<AttemptPathIdentifier,
+                AttemptPathInfo> notification) {
+              if (LOG.isDebugEnabled()) {
+                LOG.debug("PathCache Eviction: " + notification.getKey() +
+                    ", Reason=" + notification.getCause());
+              }
+            }
+          }
+      ).maximumWeight(MAX_WEIGHT).weigher(
+          new Weigher<AttemptPathIdentifier, AttemptPathInfo>() {
+            @Override
+            public int weigh(AttemptPathIdentifier key,
+                AttemptPathInfo value) {
+              return key.jobId.length() + key.user.length() +
+                  key.attemptId.length()+
+                  value.indexPath.toString().length() +
+                  value.dataPath.toString().length();
+            }
+          }
+      ).build(new CacheLoader<AttemptPathIdentifier, AttemptPathInfo>() {
+        @Override
+        public AttemptPathInfo load(AttemptPathIdentifier key) throws
+            Exception {
+          String base = getBaseLocation(key.jobId, key.user);
+          String attemptBase = base + key.attemptId;
+          Path indexFileName = lDirAlloc.getLocalPathToRead(
+              attemptBase + "/" + INDEX_FILE_NAME, conf);
+          Path mapOutputFileName = lDirAlloc.getLocalPathToRead(
+              attemptBase + "/" + DATA_FILE_NAME, conf);
+
+          if (LOG.isDebugEnabled()) {
+            LOG.debug("Loaded : " + key + " via loader");
+          }
+          return new AttemptPathInfo(indexFileName, mapOutputFileName);
+        }
+      });
 
     public Shuffle(Configuration conf) {
       this.conf = conf;
       indexCache = new IndexCache(new JobConf(conf));
       this.port = conf.getInt(SHUFFLE_PORT_CONFIG_KEY, DEFAULT_SHUFFLE_PORT);
     }
-    
+
     public void setPort(int port) {
       this.port = port;
     }
@@ -908,13 +963,8 @@ public class ShuffleHandler extends AuxiliaryService {
       Channel ch = evt.getChannel();
       String user = userRsrc.get(jobId);
 
-      // $x/$user/appcache/$appId/output/$mapId
-      // TODO: Once Shuffle is out of NM, this can use MR APIs to convert
-      // between App and Job
-      String outputBasePathStr = getBaseLocation(jobId, user);
-
       try {
-        populateHeaders(mapIds, outputBasePathStr, user, reduceId, request,
+        populateHeaders(mapIds, jobId, user, reduceId, request,
           response, keepAliveParam, mapOutputInfoMap);
       } catch(IOException e) {
         ch.write(response);
@@ -926,7 +976,7 @@ public class ShuffleHandler extends AuxiliaryService {
       ch.write(response);
       //Initialize one ReduceContext object per messageReceived call
       ReduceContext reduceContext = new ReduceContext(mapIds, reduceId, ctx,
-          user, mapOutputInfoMap, outputBasePathStr);
+          user, mapOutputInfoMap, jobId);
       for (int i = 0; i < Math.min(maxSessionOpenFiles, mapIds.size()); i++) {
         ChannelFuture nextMap = sendMap(reduceContext);
         if(nextMap == null) {
@@ -957,9 +1007,8 @@ public class ShuffleHandler extends AuxiliaryService {
         try {
           MapOutputInfo info = reduceContext.getInfoMap().get(mapId);
           if (info == null) {
-            info = getMapOutputInfo(reduceContext.getOutputBasePathStr() +
-                       mapId, mapId, reduceContext.getReduceId(),
-                       reduceContext.getUser());
+            info = getMapOutputInfo(mapId, reduceContext.getReduceId(),
+                reduceContext.getJobId(), reduceContext.getUser());
           }
           nextMap = sendMapOutput(
               reduceContext.getCtx(),
@@ -1003,46 +1052,58 @@ public class ShuffleHandler extends AuxiliaryService {
       return baseStr;
     }
 
-    protected MapOutputInfo getMapOutputInfo(String base, String mapId,
-        int reduce, String user) throws IOException {
-      // Index file
-      Path indexFileName =
-          lDirAlloc.getLocalPathToRead(base + "/file.out.index", conf);
+    protected MapOutputInfo getMapOutputInfo(String mapId, int reduce,
+        String jobId, String user) throws IOException {
+      AttemptPathInfo pathInfo;
+      try {
+        AttemptPathIdentifier identifier = new AttemptPathIdentifier(
+            jobId, user, mapId);
+        pathInfo = pathCache.get(identifier);
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("Retrieved pathInfo for " + identifier +
+              " check for corresponding loaded messages to determine whether" +
+              " it was loaded or cached");
+        }
+      } catch (ExecutionException e) {
+        if (e.getCause() instanceof IOException) {
+          throw (IOException) e.getCause();
+        } else {
+          throw new RuntimeException(e.getCause());
+        }
+      }
+
       IndexRecord info =
-          indexCache.getIndexInformation(mapId, reduce, indexFileName, user);
+        indexCache.getIndexInformation(mapId, reduce, pathInfo.indexPath, user);
 
-      Path mapOutputFileName =
-          lDirAlloc.getLocalPathToRead(base + "/file.out", conf);
       if (LOG.isDebugEnabled()) {
-        LOG.debug(base + " : " + mapOutputFileName + " : " + indexFileName);
+        LOG.debug("getMapOutputInfo: jobId=" + jobId + ", mapId=" + mapId +
+            ",dataFile=" + pathInfo.dataPath + ", indexFile=" +
+            pathInfo.indexPath);
       }
-      MapOutputInfo outputInfo = new MapOutputInfo(mapOutputFileName, info);
+
+      MapOutputInfo outputInfo = new MapOutputInfo(pathInfo.dataPath, info);
       return outputInfo;
     }
 
-    protected void populateHeaders(List<String> mapIds, String outputBaseStr,
+    protected void populateHeaders(List<String> mapIds, String jobId,
         String user, int reduce, HttpRequest request, HttpResponse response,
         boolean keepAliveParam, Map<String, MapOutputInfo> mapOutputInfoMap)
         throws IOException {
 
       long contentLength = 0;
       for (String mapId : mapIds) {
-        String base = outputBaseStr + mapId;
-        MapOutputInfo outputInfo = getMapOutputInfo(base, mapId, reduce, user);
+        MapOutputInfo outputInfo = getMapOutputInfo(mapId, reduce, jobId, user);
         if (mapOutputInfoMap.size() < mapOutputMetaInfoCacheSize) {
           mapOutputInfoMap.put(mapId, outputInfo);
         }
-        // Index file
-        Path indexFileName =
-            lDirAlloc.getLocalPathToRead(base + "/file.out.index", conf);
-        IndexRecord info =
-            indexCache.getIndexInformation(mapId, reduce, indexFileName, user);
+
         ShuffleHeader header =
-            new ShuffleHeader(mapId, info.partLength, info.rawLength, reduce);
+            new ShuffleHeader(mapId, outputInfo.indexRecord.partLength,
+            outputInfo.indexRecord.rawLength, reduce);
         DataOutputBuffer dob = new DataOutputBuffer();
         header.write(dob);
 
-        contentLength += info.partLength;
+        contentLength += outputInfo.indexRecord.partLength;
         contentLength += dob.getLength();
       }
 
@@ -1215,4 +1276,64 @@ public class ShuffleHandler extends AuxiliaryService {
       }
     }
   }
+  
+  static class AttemptPathInfo {
+    // TODO Change this over to just store local dir indices, instead of the
+    // entire path. Far more efficient.
+    private final Path indexPath;
+    private final Path dataPath;
+
+    public AttemptPathInfo(Path indexPath, Path dataPath) {
+      this.indexPath = indexPath;
+      this.dataPath = dataPath;
+    }
+  }
+
+  static class AttemptPathIdentifier {
+    private final String jobId;
+    private final String user;
+    private final String attemptId;
+
+    public AttemptPathIdentifier(String jobId, String user, String attemptId) {
+      this.jobId = jobId;
+      this.user = user;
+      this.attemptId = attemptId;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+      if (this == o) {
+        return true;
+      }
+      if (o == null || getClass() != o.getClass()) {
+        return false;
+      }
+
+      AttemptPathIdentifier that = (AttemptPathIdentifier) o;
+
+      if (!attemptId.equals(that.attemptId)) {
+        return false;
+      }
+      if (!jobId.equals(that.jobId)) {
+        return false;
+      }
+
+      return true;
+    }
+
+    @Override
+    public int hashCode() {
+      int result = jobId.hashCode();
+      result = 31 * result + attemptId.hashCode();
+      return result;
+    }
+
+    @Override
+    public String toString() {
+      return "AttemptPathIdentifier{" +
+          "attemptId='" + attemptId + '\'' +
+          ", jobId='" + jobId + '\'' +
+          '}';
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8107fcd/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-shuffle/src/test/java/org/apache/hadoop/mapred/TestShuffleHandler.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-shuffle/src/test/java/org/apache/hadoop/mapred/TestShuffleHandler.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-shuffle/src/test/java/org/apache/hadoop/mapred/TestShuffleHandler.java
index 25a622b..1717588 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-shuffle/src/test/java/org/apache/hadoop/mapred/TestShuffleHandler.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-shuffle/src/test/java/org/apache/hadoop/mapred/TestShuffleHandler.java
@@ -110,8 +110,8 @@ public class TestShuffleHandler {
             throws IOException {
         }
         @Override
-        protected MapOutputInfo getMapOutputInfo(String base, String mapId,
-            int reduce, String user) throws IOException {
+        protected MapOutputInfo getMapOutputInfo(String mapId, int reduce,
+            String jobId, String user) throws IOException {
           // Do nothing.
           return null;
         }
@@ -230,8 +230,8 @@ public class TestShuffleHandler {
         // replace the shuffle handler with one stubbed for testing
         return new Shuffle(conf) {
           @Override
-          protected MapOutputInfo getMapOutputInfo(String base, String mapId,
-              int reduce, String user) throws IOException {
+          protected MapOutputInfo getMapOutputInfo(String mapId, int reduce,
+              String jobId, String user) throws IOException {
             return null;
           }
           @Override
@@ -325,8 +325,8 @@ public class TestShuffleHandler {
         // replace the shuffle handler with one stubbed for testing
         return new Shuffle(conf) {
           @Override
-          protected MapOutputInfo getMapOutputInfo(String base, String mapId,
-              int reduce, String user) throws IOException {
+          protected MapOutputInfo getMapOutputInfo(String mapId, int reduce,
+              String jobId, String user) throws IOException {
             return null;
           }
           @Override
@@ -534,8 +534,8 @@ public class TestShuffleHandler {
         // replace the shuffle handler with one stubbed for testing
         return new Shuffle(conf) {
           @Override
-          protected MapOutputInfo getMapOutputInfo(String base, String mapId,
-              int reduce, String user) throws IOException {
+          protected MapOutputInfo getMapOutputInfo(String mapId, int reduce,
+              String jobId, String user) throws IOException {
             // Do nothing.
             return null;
           }


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


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

Branch: refs/heads/HDFS-1312
Commit: 62e4dcd2d63ab3bce0b2fd942ebba1dd82693733
Parents: 90a0326 d8107fc
Author: Anu Engineer <ae...@apache.org>
Authored: Tue Jun 21 17:38:45 2016 -0700
Committer: Anu Engineer <ae...@apache.org>
Committed: Tue Jun 21 17:38:45 2016 -0700

----------------------------------------------------------------------
 hadoop-common-project/hadoop-common/pom.xml     |   5 +
 .../hadoop/fs/FileSystemStorageStatistics.java  |  10 +-
 .../main/java/org/apache/hadoop/fs/Path.java    | 181 ++++++--
 .../fs/TestFileSystemStorageStatistics.java     | 134 ++++++
 .../hadoop/hdfs/AsyncDistributedFileSystem.java | 213 ---------
 .../hadoop/hdfs/DFSOpsCountStatistics.java      |   5 +-
 .../hadoop/hdfs/DistributedFileSystem.java      |  10 -
 .../hadoop/hdfs/protocol/CachePoolInfo.java     |  25 +
 .../hadoop/hdfs/protocolPB/PBHelperClient.java  |   7 +
 .../src/main/proto/ClientNamenodeProtocol.proto |   1 +
 .../hadoop/hdfs/TestDFSOpsCountStatistics.java  | 108 +++++
 .../src/main/conf/httpfs-env.sh                 |   4 +
 .../src/main/libexec/httpfs-config.sh           |   1 +
 .../src/main/tomcat/server.xml                  |   1 +
 .../src/main/tomcat/ssl-server.xml.conf         |   1 +
 .../src/site/markdown/ServerSetup.md.vm         |   4 +
 .../server/datanode/DataNodeFaultInjector.java  |   2 +
 .../hdfs/server/datanode/DataXceiver.java       |   3 +
 .../datanode/fsdataset/impl/FsDatasetImpl.java  |   4 +
 .../hdfs/server/namenode/CacheManager.java      |  19 +-
 .../hadoop/hdfs/server/namenode/CachePool.java  |  28 +-
 .../server/namenode/FSImageSerialization.java   |  16 +-
 .../apache/hadoop/hdfs/tools/CacheAdmin.java    |  34 +-
 .../org/apache/hadoop/hdfs/TestAsyncDFS.java    | 454 -------------------
 .../apache/hadoop/hdfs/TestAsyncDFSRename.java  | 187 --------
 .../apache/hadoop/hdfs/TestAsyncHDFSWithHA.java | 181 --------
 .../fsdataset/impl/TestSpaceReservation.java    |  48 ++
 .../src/test/resources/testCacheAdminConf.xml   |  51 ++-
 .../v2/app/webapp/TestAMWebServices.java        |  29 +-
 .../v2/app/webapp/TestAMWebServicesAttempt.java |  20 +-
 .../app/webapp/TestAMWebServicesAttempts.java   |  24 +-
 .../v2/app/webapp/TestAMWebServicesJobConf.java |  27 +-
 .../v2/app/webapp/TestAMWebServicesJobs.java    |  31 +-
 .../v2/app/webapp/TestAMWebServicesTasks.java   |  32 +-
 .../apache/hadoop/mapreduce/v2/util/MRApps.java |  70 ++-
 .../java/org/apache/hadoop/mapreduce/Job.java   |   7 +-
 .../hadoop/mapreduce/JobResourceUploader.java   |  20 +-
 .../hadoop/mapreduce/JobSubmissionFiles.java    |   4 +-
 .../apache/hadoop/mapreduce/JobSubmitter.java   |   6 +-
 .../ClientDistributedCacheManager.java          |  31 +-
 .../mapreduce/filecache/DistributedCache.java   |  76 +++-
 .../src/main/resources/mapred-default.xml       |  18 +
 .../TestClientDistributedCacheManager.java      | 151 +++++-
 .../filecache/TestDistributedCache.java         | 132 ++++++
 .../v2/hs/webapp/TestHsWebServices.java         |  28 +-
 .../v2/hs/webapp/TestHsWebServicesAttempts.java |  33 +-
 .../v2/hs/webapp/TestHsWebServicesJobConf.java  |  20 +-
 .../v2/hs/webapp/TestHsWebServicesJobs.java     |  35 +-
 .../hs/webapp/TestHsWebServicesJobsQuery.java   |  60 +--
 .../v2/hs/webapp/TestHsWebServicesTasks.java    |  32 +-
 .../hadoop/mapred/TestLocalJobSubmission.java   |  34 +-
 .../apache/hadoop/mapreduce/v2/TestMRJobs.java  |  48 +-
 .../apache/hadoop/mapred/ShuffleHandler.java    | 193 ++++++--
 .../hadoop/mapred/TestShuffleHandler.java       |  16 +-
 hadoop-project/pom.xml                          |  19 +-
 .../hadoop/fs/s3a/S3AStorageStatistics.java     |   5 +-
 .../hadoop/fs/s3a/TestS3ACredentialsInURL.java  |  16 +-
 .../apache/hadoop/yarn/client/cli/LogsCLI.java  |   3 +-
 .../hadoop-yarn/hadoop-yarn-common/pom.xml      |   6 +
 .../yarn/client/api/impl/TimelineWriter.java    |   3 +-
 .../client/api/impl/TestTimelineClient.java     |   4 +-
 .../api/impl/TestTimelineClientForATS1_5.java   |   4 +-
 .../hadoop/yarn/webapp/GuiceServletConfig.java  |  43 ++
 .../yarn/webapp/WebServicesTestUtils.java       |  11 +
 .../webapp/TestAHSWebServices.java              |  64 ++-
 .../webapp/TestTimelineWebServices.java         |  56 ++-
 .../server/nodemanager/ContainerExecutor.java   |  15 +-
 .../nodemanager/webapp/TestNMWebServices.java   |  29 +-
 .../webapp/TestNMWebServicesApps.java           |  42 +-
 .../webapp/TestNMWebServicesContainers.java     |  29 +-
 .../webapp/TestRMWebServices.java               |  29 +-
 .../webapp/TestRMWebServicesApps.java           |  51 ++-
 .../TestRMWebServicesAppsModification.java      |  89 ++--
 .../webapp/TestRMWebServicesCapacitySched.java  |  65 ++-
 .../TestRMWebServicesDelegationTokens.java      |  66 ++-
 .../webapp/TestRMWebServicesFairScheduler.java  |  29 +-
 .../TestRMWebServicesForCSWithPartitions.java   |  92 ++--
 .../webapp/TestRMWebServicesNodeLabels.java     |  30 +-
 .../webapp/TestRMWebServicesNodes.java          |  35 +-
 .../webapp/TestRMWebServicesReservation.java    | 139 +++---
 .../yarn/webapp/TestRMWithCSRFFilter.java       |  22 +-
 .../hadoop/yarn/webapp/TestRMWithXFSFilter.java |  19 +-
 82 files changed, 2015 insertions(+), 1914 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/62e4dcd2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java
----------------------------------------------------------------------


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


[15/17] hadoop git commit: HADOOP-13287. TestS3ACredentials#testInstantiateFromURL fails if AWS secret key contains +. Contributed by Chris Nauroth.

Posted by ae...@apache.org.
HADOOP-13287. TestS3ACredentials#testInstantiateFromURL fails if AWS secret key contains +. Contributed by Chris Nauroth.


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

Branch: refs/heads/HDFS-1312
Commit: b2c596cdda7c129951074bc53b4b9ecfedbf080a
Parents: 605b4b6
Author: Chris Nauroth <cn...@apache.org>
Authored: Tue Jun 21 11:28:52 2016 -0700
Committer: Chris Nauroth <cn...@apache.org>
Committed: Tue Jun 21 11:28:52 2016 -0700

----------------------------------------------------------------------
 .../hadoop/fs/s3a/TestS3ACredentialsInURL.java      | 16 +++++++++-------
 1 file changed, 9 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/b2c596cd/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
index f1da72b..8cb7c0f 100644
--- 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
@@ -33,7 +33,6 @@ 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;
 
@@ -73,7 +72,11 @@ public class TestS3ACredentialsInURL extends Assert {
         accessKey, secretKey);
     if (secretKey.contains("/")) {
       assertTrue("test URI encodes the / symbol", secretsURI.toString().
-          contains("%2F"));
+          contains("%252F"));
+    }
+    if (secretKey.contains("+")) {
+      assertTrue("test URI encodes the + symbol", secretsURI.toString().
+          contains("%252B"));
     }
     assertFalse("Does not contain secrets", original.equals(secretsURI));
 
@@ -132,8 +135,7 @@ public class TestS3ACredentialsInURL extends Assert {
 
   private URI createUriWithEmbeddedSecrets(URI original,
       String accessKey,
-      String secretKey) throws URISyntaxException,
-      UnsupportedEncodingException {
+      String secretKey) throws UnsupportedEncodingException {
     String encodedSecretKey = URLEncoder.encode(secretKey, "UTF-8");
     String formattedString = String.format("%s://%s:%s@%s/%s/",
         original.getScheme(),
@@ -143,10 +145,10 @@ public class TestS3ACredentialsInURL extends Assert {
         original.getPath());
     URI testURI;
     try {
-      testURI = new URI(formattedString);
-    } catch (URISyntaxException e) {
+      testURI = new Path(formattedString).toUri();
+    } catch (IllegalArgumentException e) {
       // inner cause is stripped to keep any secrets out of stack traces
-      throw new URISyntaxException("", "Could not encode URI");
+      throw new IllegalArgumentException("Could not encode Path");
     }
     return testURI;
   }


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


[04/17] hadoop git commit: HDFS-10423. Increase default value of httpfs maxHttpHeaderSize. Contributed by Nicolae Popa.

Posted by ae...@apache.org.
HDFS-10423. Increase default value of httpfs maxHttpHeaderSize. Contributed by Nicolae Popa.


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

Branch: refs/heads/HDFS-1312
Commit: aa1b583cf99d1a7cfe554d1769fc4af252374663
Parents: 5370a6f
Author: Aaron T. Myers <at...@apache.org>
Authored: Mon Jun 20 13:46:11 2016 -0700
Committer: Aaron T. Myers <at...@apache.org>
Committed: Mon Jun 20 13:46:56 2016 -0700

----------------------------------------------------------------------
 .../hadoop-hdfs-httpfs/src/main/conf/httpfs-env.sh               | 4 ++++
 .../hadoop-hdfs-httpfs/src/main/libexec/httpfs-config.sh         | 1 +
 .../hadoop-hdfs-httpfs/src/main/tomcat/server.xml                | 1 +
 .../hadoop-hdfs-httpfs/src/main/tomcat/ssl-server.xml.conf       | 1 +
 .../hadoop-hdfs-httpfs/src/site/markdown/ServerSetup.md.vm       | 4 ++++
 5 files changed, 11 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/aa1b583c/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/conf/httpfs-env.sh
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/conf/httpfs-env.sh b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/conf/httpfs-env.sh
index f012453..300d2ac 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/conf/httpfs-env.sh
+++ b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/conf/httpfs-env.sh
@@ -38,6 +38,10 @@
 #
 # export HTTPFS_HTTP_HOSTNAME=$(hostname -f)
 
+# The maximum size of Tomcat HTTP header
+#
+# export HTTPFS_MAX_HTTP_HEADER_SIZE=65536
+
 # The location of the SSL keystore if using SSL
 #
 # export HTTPFS_SSL_KEYSTORE_FILE=${HOME}/.keystore

http://git-wip-us.apache.org/repos/asf/hadoop/blob/aa1b583c/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/libexec/httpfs-config.sh
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/libexec/httpfs-config.sh b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/libexec/httpfs-config.sh
index ba4b406..176dd32 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/libexec/httpfs-config.sh
+++ b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/libexec/httpfs-config.sh
@@ -42,6 +42,7 @@ function hadoop_subproject_init
   export HADOOP_CATALINA_HTTP_PORT="${HTTPFS_HTTP_PORT:-14000}"
   export HADOOP_CATALINA_ADMIN_PORT="${HTTPFS_ADMIN_PORT:-$((HADOOP_CATALINA_HTTP_PORT+1))}"
   export HADOOP_CATALINA_MAX_THREADS="${HTTPFS_MAX_THREADS:-150}"
+  export HADOOP_CATALINA_MAX_HTTP_HEADER_SIZE="${HTTPFS_MAX_HTTP_HEADER_SIZE:-65536}"
 
   export HTTPFS_SSL_ENABLED=${HTTPFS_SSL_ENABLED:-false}
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/aa1b583c/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/tomcat/server.xml
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/tomcat/server.xml b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/tomcat/server.xml
index a425bdd..67f2159 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/tomcat/server.xml
+++ b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/tomcat/server.xml
@@ -71,6 +71,7 @@
     -->
     <Connector port="${httpfs.http.port}" protocol="HTTP/1.1"
                connectionTimeout="20000"
+               maxHttpHeaderSize="${httpfs.max.http.header.size}"
                redirectPort="8443"/>
     <!-- A "Connector" using the shared thread pool-->
     <!--

http://git-wip-us.apache.org/repos/asf/hadoop/blob/aa1b583c/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/tomcat/ssl-server.xml.conf
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/tomcat/ssl-server.xml.conf b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/tomcat/ssl-server.xml.conf
index 124a5e4..d53431d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/tomcat/ssl-server.xml.conf
+++ b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/tomcat/ssl-server.xml.conf
@@ -70,6 +70,7 @@
          described in the APR documentation -->
     <Connector port="${httpfs.http.port}" protocol="HTTP/1.1" SSLEnabled="true"
                maxThreads="150" scheme="https" secure="true"
+               maxHttpHeaderSize="${httpfs.max.http.header.size}"
                clientAuth="false" sslEnabledProtocols="TLSv1,TLSv1.1,TLSv1.2,SSLv2Hello"
                keystoreFile="${httpfs.ssl.keystore.file}"
                keystorePass="_httpfs_ssl_keystore_pass_"/>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/aa1b583c/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/site/markdown/ServerSetup.md.vm
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/site/markdown/ServerSetup.md.vm b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/site/markdown/ServerSetup.md.vm
index 3c7f9d3..6a03a45 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/site/markdown/ServerSetup.md.vm
+++ b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/site/markdown/ServerSetup.md.vm
@@ -80,6 +80,8 @@ HttpFS preconfigures the HTTP and Admin ports in Tomcat's `server.xml` to 14000
 
 Tomcat logs are also preconfigured to go to HttpFS's `logs/` directory.
 
+HttpFS default value for the maxHttpHeaderSize parameter in Tomcat's `server.xml` is set to 65536 by default.
+
 The following environment variables (which can be set in HttpFS's `etc/hadoop/httpfs-env.sh` script) can be used to alter those values:
 
 * HTTPFS\_HTTP\_PORT
@@ -88,6 +90,8 @@ The following environment variables (which can be set in HttpFS's `etc/hadoop/ht
 
 * HADOOP\_LOG\_DIR
 
+* HTTPFS\_MAX\_HTTP\_HEADER\_SIZE
+
 HttpFS Configuration
 --------------------
 


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


[13/17] hadoop git commit: HADOOP-13296. Cleanup javadoc for Path. Contributed by Daniel Templeton.

Posted by ae...@apache.org.
HADOOP-13296. Cleanup javadoc for Path. Contributed by Daniel Templeton.


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

Branch: refs/heads/HDFS-1312
Commit: e15cd43369eb6d478844f25897e4a86065c62168
Parents: f2ac132
Author: Akira Ajisaka <aa...@apache.org>
Authored: Tue Jun 21 20:16:22 2016 +0900
Committer: Akira Ajisaka <aa...@apache.org>
Committed: Tue Jun 21 20:16:22 2016 +0900

----------------------------------------------------------------------
 .../main/java/org/apache/hadoop/fs/Path.java    | 181 +++++++++++++++----
 1 file changed, 142 insertions(+), 39 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/e15cd433/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/Path.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/Path.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/Path.java
index a38a46c..f18a675 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/Path.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/Path.java
@@ -30,7 +30,8 @@ import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
 
-/** Names a file or directory in a {@link FileSystem}.
+/**
+ * Names a file or directory in a {@link FileSystem}.
  * Path strings use slash as the directory separator.
  */
 @Stringable
@@ -38,26 +39,37 @@ import org.apache.hadoop.conf.Configuration;
 @InterfaceStability.Stable
 public class Path implements Comparable {
 
-  /** The directory separator, a slash. */
+  /**
+   * The directory separator, a slash.
+   */
   public static final String SEPARATOR = "/";
+
+  /**
+   * The directory separator, a slash, as a character.
+   */
   public static final char SEPARATOR_CHAR = '/';
   
+  /**
+   * The current directory, ".".
+   */
   public static final String CUR_DIR = ".";
   
-  public static final boolean WINDOWS
-    = System.getProperty("os.name").startsWith("Windows");
+  /**
+   * Whether the current host is a Windows machine.
+   */
+  public static final boolean WINDOWS =
+      System.getProperty("os.name").startsWith("Windows");
 
   /**
    *  Pre-compiled regular expressions to detect path formats.
    */
-  private static final Pattern hasUriScheme =
-      Pattern.compile("^[a-zA-Z][a-zA-Z0-9+-.]+:");
-  private static final Pattern hasDriveLetterSpecifier =
+  private static final Pattern HAS_DRIVE_LETTER_SPECIFIER =
       Pattern.compile("^/?[a-zA-Z]:");
 
-  private URI uri;                                // a hierarchical uri
+  private URI uri; // a hierarchical uri
 
   /**
+   * Test whether this Path uses a scheme and is relative.
    * Pathnames with scheme and relative path are illegal.
    */
   void checkNotSchemeWithRelative() {
@@ -73,6 +85,12 @@ public class Path implements Comparable {
     }
   }
 
+  /**
+   * Return a version of the given Path without the scheme information.
+   *
+   * @param path the source Path
+   * @return a copy of this Path without the scheme information
+   */
   public static Path getPathWithoutSchemeAndAuthority(Path path) {
     // This code depends on Path.toString() to remove the leading slash before
     // the drive specification on Windows.
@@ -82,22 +100,42 @@ public class Path implements Comparable {
     return newPath;
   }
 
-  /** Resolve a child path against a parent path. */
+  /**
+   * Create a new Path based on the child path resolved against the parent path.
+   *
+   * @param parent the parent path
+   * @param child the child path
+   */
   public Path(String parent, String child) {
     this(new Path(parent), new Path(child));
   }
 
-  /** Resolve a child path against a parent path. */
+  /**
+   * Create a new Path based on the child path resolved against the parent path.
+   *
+   * @param parent the parent path
+   * @param child the child path
+   */
   public Path(Path parent, String child) {
     this(parent, new Path(child));
   }
 
-  /** Resolve a child path against a parent path. */
+  /**
+   * Create a new Path based on the child path resolved against the parent path.
+   *
+   * @param parent the parent path
+   * @param child the child path
+   */
   public Path(String parent, Path child) {
     this(new Path(parent), child);
   }
 
-  /** Resolve a child path against a parent path. */
+  /**
+   * Create a new Path based on the child path resolved against the parent path.
+   *
+   * @param parent the parent path
+   * @param child the child path
+   */
   public Path(Path parent, Path child) {
     // Add a slash to parent's path so resolution is compatible with URI's
     URI parentUri = parent.uri;
@@ -127,8 +165,12 @@ public class Path implements Comparable {
     }   
   }
   
-  /** Construct a path from a String.  Path strings are URIs, but with
-   * unescaped elements and some additional normalization. */
+  /**
+   * Construct a path from a String.  Path strings are URIs, but with
+   * unescaped elements and some additional normalization.
+   *
+   * @param pathString the path string
+   */
   public Path(String pathString) throws IllegalArgumentException {
     checkPathArg( pathString );
     
@@ -172,12 +214,20 @@ public class Path implements Comparable {
 
   /**
    * Construct a path from a URI
+   *
+   * @param aUri the source URI
    */
   public Path(URI aUri) {
     uri = aUri.normalize();
   }
   
-  /** Construct a Path from components. */
+  /**
+   * Construct a Path from components.
+   *
+   * @param scheme the scheme
+   * @param authority the authority
+   * @param path the path
+   */
   public Path(String scheme, String authority, String path) {
     checkPathArg( path );
 
@@ -210,9 +260,9 @@ public class Path implements Comparable {
    * The returned path has the scheme and authority of the first path.  On
    * Windows, the drive specification in the second path is discarded.
    * 
-   * @param path1 Path first path
-   * @param path2 Path second path, to be appended relative to path1
-   * @return Path merged path
+   * @param path1 the first path
+   * @param path2 the second path, to be appended relative to path1
+   * @return the merged path
    */
   public static Path mergePaths(Path path1, Path path2) {
     String path2Str = path2.toUri().getPath();
@@ -228,10 +278,11 @@ public class Path implements Comparable {
   /**
    * Normalize a path string to use non-duplicated forward slashes as
    * the path separator and remove any trailing path separators.
-   * @param scheme Supplies the URI scheme. Used to deduce whether we
-   *               should replace backslashes or not.
-   * @param path Supplies the scheme-specific part
-   * @return Normalized path string.
+   *
+   * @param scheme the URI scheme. Used to deduce whether we
+   * should replace backslashes or not
+   * @param path the scheme-specific part
+   * @return the normalized path string
    */
   private static String normalizePath(String scheme, String path) {
     // Remove double forward slashes.
@@ -257,7 +308,7 @@ public class Path implements Comparable {
   }
 
   private static boolean hasWindowsDrive(String path) {
-    return (WINDOWS && hasDriveLetterSpecifier.matcher(path).find());
+    return (WINDOWS && HAS_DRIVE_LETTER_SPECIFIER.matcher(path).find());
   }
 
   private static int startPositionWithoutWindowsDrive(String path) {
@@ -272,10 +323,10 @@ public class Path implements Comparable {
    * Determine whether a given path string represents an absolute path on
    * Windows. e.g. "C:/a/b" is an absolute path. "C:a/b" is not.
    *
-   * @param pathString Supplies the path string to evaluate.
-   * @param slashed true if the given path is prefixed with "/".
+   * @param pathString the path string to evaluate
+   * @param slashed true if the given path is prefixed with "/"
    * @return true if the supplied path looks like an absolute path with a Windows
-   * drive-specifier.
+   * drive-specifier
    */
   public static boolean isWindowsAbsolutePath(final String pathString,
                                               final boolean slashed) {
@@ -286,17 +337,32 @@ public class Path implements Comparable {
             (pathString.charAt(start) == '\\'));
   }
 
-  /** Convert this to a URI. */
+  /**
+   * Convert this Path to a URI.
+   *
+   * @return this Path as a URI
+   */
   public URI toUri() { return uri; }
 
-  /** Return the FileSystem that owns this Path. */
+  /**
+   * Return the FileSystem that owns this Path.
+   *
+   * @param conf the configuration to use when resolving the FileSystem
+   * @return the FileSystem that owns this Path
+   * @throws java.io.IOException thrown if there's an issue resolving the
+   * FileSystem
+   */
   public FileSystem getFileSystem(Configuration conf) throws IOException {
     return FileSystem.get(this.toUri(), conf);
   }
 
   /**
-   * Is an absolute path (ie a slash relative path part)
-   *  AND  a scheme is null AND  authority is null.
+   * Returns true if the path component (i.e. directory) of this URI is
+   * absolute <strong>and</strong> the scheme is null, <b>and</b> the authority
+   * is null.
+   *
+   * @return whether the path is absolute and the URI has no scheme nor
+   * authority parts
    */
   public boolean isAbsoluteAndSchemeAuthorityNull() {
     return  (isUriPathAbsolute() && 
@@ -304,33 +370,50 @@ public class Path implements Comparable {
   }
   
   /**
-   *  True if the path component (i.e. directory) of this URI is absolute.
+   * Returns true if the path component (i.e. directory) of this URI is
+   * absolute.
+   *
+   * @return whether this URI's path is absolute
    */
   public boolean isUriPathAbsolute() {
     int start = startPositionWithoutWindowsDrive(uri.getPath());
     return uri.getPath().startsWith(SEPARATOR, start);
    }
   
-  /** True if the path is not a relative path and starts with root. */
+  /**
+   * Returns true if the path component (i.e. directory) of this URI is
+   * absolute.  This method is a wrapper for {@link #isUriPathAbsolute()}.
+   *
+   * @return whether this URI's path is absolute
+   */
   public boolean isAbsolute() {
      return isUriPathAbsolute();
   }
 
   /**
+   * Returns true if and only if this path represents the root of a file system.
+   *
    * @return true if and only if this path represents the root of a file system
    */
   public boolean isRoot() {
     return getParent() == null;
   }
 
-  /** Returns the final component of this path.*/
+  /**
+   * Returns the final component of this path.
+   *
+   * @return the final component of this path
+   */
   public String getName() {
     String path = uri.getPath();
     int slash = path.lastIndexOf(SEPARATOR);
     return path.substring(slash+1);
   }
 
-  /** Returns the parent of a path or null if at root. */
+  /**
+   * Returns the parent of a path or null if at root.
+   * @return the parent of a path or null if at root
+   */
   public Path getParent() {
     String path = uri.getPath();
     int lastSlash = path.lastIndexOf('/');
@@ -348,7 +431,12 @@ public class Path implements Comparable {
     return new Path(uri.getScheme(), uri.getAuthority(), parent);
   }
 
-  /** Adds a suffix to the final name in the path.*/
+  /**
+   * Adds a suffix to the final name in the path.
+   *
+   * @param suffix the suffix to add
+   * @return a new path with the suffix added
+   */
   public Path suffix(String suffix) {
     return new Path(getParent(), getName()+suffix);
   }
@@ -402,7 +490,10 @@ public class Path implements Comparable {
     return this.uri.compareTo(that.uri);
   }
   
-  /** Return the number of elements in this path. */
+  /**
+   * Returns the number of elements in this path.
+   * @return the number of elements in this path
+   */
   public int depth() {
     String path = uri.getPath();
     int depth = 0;
@@ -415,16 +506,28 @@ public class Path implements Comparable {
   }
 
   /**
-   *  Returns a qualified path object.
+   * Returns a qualified path object for the {@link FileSystem}'s working
+   * directory.
    *  
-   *  Deprecated - use {@link #makeQualified(URI, Path)}
+   * @param fs the target FileSystem
+   * @return a qualified path object for the FileSystem's working directory
+   * @deprecated use {@link #makeQualified(URI, Path)}
    */
   @Deprecated
   public Path makeQualified(FileSystem fs) {
     return makeQualified(fs.getUri(), fs.getWorkingDirectory());
   }
   
-  /** Returns a qualified path object. */
+  /**
+   * Returns a qualified path object.
+   *
+   * @param defaultUri if this path is missing the scheme or authority
+   * components, borrow them from this URI
+   * @param workingDir if this path isn't absolute, treat it as relative to this
+   * working directory
+   * @return this path if it contains a scheme and authority and is absolute, or
+   * a new path that includes a path and authority and is fully qualified
+   */
   @InterfaceAudience.LimitedPrivate({"HDFS", "MapReduce"})
   public Path makeQualified(URI defaultUri, Path workingDir ) {
     Path path = this;


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


[06/17] hadoop git commit: HADOOP-9613. [JDK8] Update jersey version to latest 1.x release.

Posted by ae...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/5d58858b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesFairScheduler.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/TestRMWebServicesFairScheduler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesFairScheduler.java
index 1aed3a6..cbe360a 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesFairScheduler.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesFairScheduler.java
@@ -30,15 +30,15 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.FairScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.QueueManager;
 import org.apache.hadoop.yarn.webapp.GenericExceptionHandler;
+import org.apache.hadoop.yarn.webapp.GuiceServletConfig;
 import org.apache.hadoop.yarn.webapp.JerseyTestBase;
 import org.codehaus.jettison.json.JSONArray;
 import org.codehaus.jettison.json.JSONException;
 import org.codehaus.jettison.json.JSONObject;
+import org.junit.Before;
 import org.junit.Test;
 
 import com.google.inject.Guice;
-import com.google.inject.Injector;
-import com.google.inject.servlet.GuiceServletContextListener;
 import com.google.inject.servlet.ServletModule;
 import com.sun.jersey.api.client.ClientResponse;
 import com.sun.jersey.api.client.WebResource;
@@ -47,9 +47,9 @@ import com.sun.jersey.test.framework.WebAppDescriptor;
 
 public class TestRMWebServicesFairScheduler extends JerseyTestBase {
   private static MockRM rm;
-  private YarnConfiguration conf;
+  private static YarnConfiguration conf;
   
-  private Injector injector = Guice.createInjector(new ServletModule() {
+  private static class WebServletModule extends ServletModule {
     @Override
     protected void configureServlets() {
       bind(JAXBContextResolver.class);
@@ -62,16 +62,21 @@ public class TestRMWebServicesFairScheduler extends JerseyTestBase {
       bind(ResourceManager.class).toInstance(rm);
       serve("/*").with(GuiceContainer.class);
     }
-  });
-  
-  public class GuiceServletConfig extends GuiceServletContextListener {
+  }
 
-    @Override
-    protected Injector getInjector() {
-      return injector;
-    }
+  static {
+    GuiceServletConfig.setInjector(
+        Guice.createInjector(new WebServletModule()));
   }
-  
+
+  @Before
+  @Override
+  public void setUp() throws Exception {
+    super.setUp();
+    GuiceServletConfig.setInjector(
+        Guice.createInjector(new WebServletModule()));
+  }
+
   public TestRMWebServicesFairScheduler() {
     super(new WebAppDescriptor.Builder(
         "org.apache.hadoop.yarn.server.resourcemanager.webapp")

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5d58858b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesForCSWithPartitions.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/TestRMWebServicesForCSWithPartitions.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesForCSWithPartitions.java
index 8087e33..046cf8a 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesForCSWithPartitions.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesForCSWithPartitions.java
@@ -38,6 +38,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration;
 import org.apache.hadoop.yarn.webapp.GenericExceptionHandler;
+import org.apache.hadoop.yarn.webapp.GuiceServletConfig;
 import org.apache.hadoop.yarn.webapp.JerseyTestBase;
 import org.apache.hadoop.yarn.webapp.WebServicesTestUtils;
 import org.codehaus.jettison.json.JSONArray;
@@ -53,8 +54,6 @@ import org.xml.sax.InputSource;
 
 import com.google.common.collect.ImmutableSet;
 import com.google.inject.Guice;
-import com.google.inject.Injector;
-import com.google.inject.servlet.GuiceServletContextListener;
 import com.google.inject.servlet.ServletModule;
 import com.sun.jersey.api.client.ClientResponse;
 import com.sun.jersey.api.client.WebResource;
@@ -78,10 +77,10 @@ public class TestRMWebServicesForCSWithPartitions extends JerseyTestBase {
   private static final ImmutableSet<String> CLUSTER_LABELS =
       ImmutableSet.of(LABEL_LX, LABEL_LY, DEFAULT_PARTITION);
   private static MockRM rm;
-  private CapacitySchedulerConfiguration csConf;
-  private YarnConfiguration conf;
+  static private CapacitySchedulerConfiguration csConf;
+  static private YarnConfiguration conf;
 
-  private Injector injector = Guice.createInjector(new ServletModule() {
+  private static class WebServletModule extends ServletModule {
     @Override
     protected void configureServlets() {
       bind(JAXBContextResolver.class);
@@ -106,76 +105,79 @@ public class TestRMWebServicesForCSWithPartitions extends JerseyTestBase {
       bind(ResourceManager.class).toInstance(rm);
       serve("/*").with(GuiceContainer.class);
     }
-  });
+  };
 
-  public class GuiceServletConfig extends GuiceServletContextListener {
-    @Override
-    protected Injector getInjector() {
-      return injector;
-    }
+  static {
+    GuiceServletConfig.setInjector(
+        Guice.createInjector(new WebServletModule()));
   }
 
   private static void setupQueueConfiguration(
-      CapacitySchedulerConfiguration conf, ResourceManager rm) {
+      CapacitySchedulerConfiguration config, ResourceManager resourceManager) {
 
     // Define top-level queues
-    conf.setQueues(CapacitySchedulerConfiguration.ROOT,
+    config.setQueues(CapacitySchedulerConfiguration.ROOT,
         new String[] { QUEUE_A, QUEUE_B, QUEUE_C });
     String interMediateQueueC =
         CapacitySchedulerConfiguration.ROOT + "." + QUEUE_C;
-    conf.setQueues(interMediateQueueC,
+    config.setQueues(interMediateQueueC,
         new String[] { LEAF_QUEUE_C1, LEAF_QUEUE_C2 });
-    conf.setCapacityByLabel(CapacitySchedulerConfiguration.ROOT, LABEL_LX, 100);
-    conf.setCapacityByLabel(CapacitySchedulerConfiguration.ROOT, LABEL_LY, 100);
+    config.setCapacityByLabel(
+        CapacitySchedulerConfiguration.ROOT, LABEL_LX, 100);
+    config.setCapacityByLabel(
+        CapacitySchedulerConfiguration.ROOT, LABEL_LY, 100);
 
     String leafQueueA = CapacitySchedulerConfiguration.ROOT + "." + QUEUE_A;
-    conf.setCapacity(leafQueueA, 30);
-    conf.setMaximumCapacity(leafQueueA, 50);
+    config.setCapacity(leafQueueA, 30);
+    config.setMaximumCapacity(leafQueueA, 50);
 
     String leafQueueB = CapacitySchedulerConfiguration.ROOT + "." + QUEUE_B;
-    conf.setCapacity(leafQueueB, 30);
-    conf.setMaximumCapacity(leafQueueB, 50);
+    config.setCapacity(leafQueueB, 30);
+    config.setMaximumCapacity(leafQueueB, 50);
 
-    conf.setCapacity(interMediateQueueC, 40);
-    conf.setMaximumCapacity(interMediateQueueC, 50);
+    config.setCapacity(interMediateQueueC, 40);
+    config.setMaximumCapacity(interMediateQueueC, 50);
 
     String leafQueueC1 = interMediateQueueC + "." + LEAF_QUEUE_C1;
-    conf.setCapacity(leafQueueC1, 50);
-    conf.setMaximumCapacity(leafQueueC1, 60);
+    config.setCapacity(leafQueueC1, 50);
+    config.setMaximumCapacity(leafQueueC1, 60);
 
     String leafQueueC2 = interMediateQueueC + "." + LEAF_QUEUE_C2;
-    conf.setCapacity(leafQueueC2, 50);
-    conf.setMaximumCapacity(leafQueueC2, 70);
+    config.setCapacity(leafQueueC2, 50);
+    config.setMaximumCapacity(leafQueueC2, 70);
 
     // Define label specific configuration
-    conf.setAccessibleNodeLabels(leafQueueA, ImmutableSet.of(DEFAULT_PARTITION));
-    conf.setAccessibleNodeLabels(leafQueueB, ImmutableSet.of(LABEL_LX));
-    conf.setAccessibleNodeLabels(interMediateQueueC,
+    config.setAccessibleNodeLabels(
+        leafQueueA, ImmutableSet.of(DEFAULT_PARTITION));
+    config.setAccessibleNodeLabels(leafQueueB, ImmutableSet.of(LABEL_LX));
+    config.setAccessibleNodeLabels(interMediateQueueC,
         ImmutableSet.of(LABEL_LX, LABEL_LY));
-    conf.setAccessibleNodeLabels(leafQueueC1,
+    config.setAccessibleNodeLabels(leafQueueC1,
         ImmutableSet.of(LABEL_LX, LABEL_LY));
-    conf.setAccessibleNodeLabels(leafQueueC2,
+    config.setAccessibleNodeLabels(leafQueueC2,
         ImmutableSet.of(LABEL_LX, LABEL_LY));
-    conf.setDefaultNodeLabelExpression(leafQueueB, LABEL_LX);
-    conf.setDefaultNodeLabelExpression(leafQueueC1, LABEL_LX);
-    conf.setDefaultNodeLabelExpression(leafQueueC2, LABEL_LY);
-
-    conf.setCapacityByLabel(leafQueueB, LABEL_LX, 30);
-    conf.setCapacityByLabel(interMediateQueueC, LABEL_LX, 70);
-    conf.setCapacityByLabel(leafQueueC1, LABEL_LX, 40);
-    conf.setCapacityByLabel(leafQueueC2, LABEL_LX, 60);
-
-    conf.setCapacityByLabel(interMediateQueueC, LABEL_LY, 100);
-    conf.setCapacityByLabel(leafQueueC1, LABEL_LY, 50);
-    conf.setCapacityByLabel(leafQueueC2, LABEL_LY, 50);
-    conf.setMaximumCapacityByLabel(leafQueueC1, LABEL_LY, 75);
-    conf.setMaximumCapacityByLabel(leafQueueC2, LABEL_LY, 75);
+    config.setDefaultNodeLabelExpression(leafQueueB, LABEL_LX);
+    config.setDefaultNodeLabelExpression(leafQueueC1, LABEL_LX);
+    config.setDefaultNodeLabelExpression(leafQueueC2, LABEL_LY);
+
+    config.setCapacityByLabel(leafQueueB, LABEL_LX, 30);
+    config.setCapacityByLabel(interMediateQueueC, LABEL_LX, 70);
+    config.setCapacityByLabel(leafQueueC1, LABEL_LX, 40);
+    config.setCapacityByLabel(leafQueueC2, LABEL_LX, 60);
+
+    config.setCapacityByLabel(interMediateQueueC, LABEL_LY, 100);
+    config.setCapacityByLabel(leafQueueC1, LABEL_LY, 50);
+    config.setCapacityByLabel(leafQueueC2, LABEL_LY, 50);
+    config.setMaximumCapacityByLabel(leafQueueC1, LABEL_LY, 75);
+    config.setMaximumCapacityByLabel(leafQueueC2, LABEL_LY, 75);
   }
 
   @Before
   @Override
   public void setUp() throws Exception {
     super.setUp();
+    GuiceServletConfig.setInjector(
+        Guice.createInjector(new WebServletModule()));
   }
 
   public TestRMWebServicesForCSWithPartitions() {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5d58858b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesNodeLabels.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/TestRMWebServicesNodeLabels.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesNodeLabels.java
index 7a72cb1..0cfaac1 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesNodeLabels.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesNodeLabels.java
@@ -41,16 +41,16 @@ import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.NodeToLabelsEntr
 import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.NodeToLabelsInfo;
 import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.NodeToLabelsEntryList;
 import org.apache.hadoop.yarn.webapp.GenericExceptionHandler;
+import org.apache.hadoop.yarn.webapp.GuiceServletConfig;
 import org.apache.hadoop.yarn.webapp.JerseyTestBase;
 import org.apache.hadoop.yarn.webapp.WebServicesTestUtils;
 import org.codehaus.jettison.json.JSONException;
 import org.codehaus.jettison.json.JSONObject;
 import org.junit.Assert;
+import org.junit.Before;
 import org.junit.Test;
 
 import com.google.inject.Guice;
-import com.google.inject.Injector;
-import com.google.inject.servlet.GuiceServletContextListener;
 import com.google.inject.servlet.ServletModule;
 import com.sun.jersey.api.client.ClientResponse;
 import com.sun.jersey.api.client.UniformInterfaceException;
@@ -69,13 +69,13 @@ public class TestRMWebServicesNodeLabels extends JerseyTestBase {
       .getLog(TestRMWebServicesNodeLabels.class);
 
   private static MockRM rm;
-  private YarnConfiguration conf;
+  private static YarnConfiguration conf;
 
-  private String userName;
-  private String notUserName;
-  private RMWebServices rmWebService;
+  private static String userName;
+  private static String notUserName;
+  private static RMWebServices rmWebService;
 
-  private Injector injector = Guice.createInjector(new ServletModule() {
+  private static class WebServletModule extends ServletModule {
 
     @Override
     protected void configureServlets() {
@@ -98,14 +98,14 @@ public class TestRMWebServicesNodeLabels extends JerseyTestBase {
           TestRMWebServicesAppsModification.TestRMCustomAuthFilter.class);
       serve("/*").with(GuiceContainer.class);
     }
-  });
-
-  public class GuiceServletConfig extends GuiceServletContextListener {
-
-    @Override
-    protected Injector getInjector() {
-      return injector;
-    }
+  };
+
+  @Override
+  @Before
+  public void setUp() throws Exception {
+    super.setUp();
+    GuiceServletConfig.setInjector(
+        Guice.createInjector(new WebServletModule()));
   }
 
   public TestRMWebServicesNodeLabels() {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5d58858b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesNodes.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/TestRMWebServicesNodes.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesNodes.java
index 5bcb0e0..718091f 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesNodes.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesNodes.java
@@ -18,6 +18,7 @@
 
 package org.apache.hadoop.yarn.server.resourcemanager.webapp;
 
+import static org.apache.hadoop.yarn.webapp.WebServicesTestUtils.assertResponseStatusCode;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
@@ -51,6 +52,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNodeRepo
 import org.apache.hadoop.yarn.util.RackResolver;
 import org.apache.hadoop.yarn.util.YarnVersionInfo;
 import org.apache.hadoop.yarn.webapp.GenericExceptionHandler;
+import org.apache.hadoop.yarn.webapp.GuiceServletConfig;
 import org.apache.hadoop.yarn.webapp.JerseyTestBase;
 import org.apache.hadoop.yarn.webapp.WebServicesTestUtils;
 import org.codehaus.jettison.json.JSONArray;
@@ -65,8 +67,6 @@ import org.xml.sax.InputSource;
 
 import com.google.common.base.Joiner;
 import com.google.inject.Guice;
-import com.google.inject.Injector;
-import com.google.inject.servlet.GuiceServletContextListener;
 import com.google.inject.servlet.ServletModule;
 import com.sun.jersey.api.client.ClientResponse;
 import com.sun.jersey.api.client.ClientResponse.Status;
@@ -79,7 +79,7 @@ public class TestRMWebServicesNodes extends JerseyTestBase {
 
   private static MockRM rm;
 
-  private Injector injector = Guice.createInjector(new ServletModule() {
+  private static class WebServletModule extends ServletModule {
     @Override
     protected void configureServlets() {
       bind(JAXBContextResolver.class);
@@ -91,20 +91,19 @@ public class TestRMWebServicesNodes extends JerseyTestBase {
       bind(ResourceManager.class).toInstance(rm);
       serve("/*").with(GuiceContainer.class);
     }
-  });
-
-  public class GuiceServletConfig extends GuiceServletContextListener {
+  }
 
-    @Override
-    protected Injector getInjector() {
-      return injector;
-    }
+  static {
+    GuiceServletConfig.setInjector(
+        Guice.createInjector(new WebServletModule()));
   }
 
   @Before
   @Override
   public void setUp() throws Exception {
     super.setUp();
+    GuiceServletConfig.setInjector(
+        Guice.createInjector(new WebServletModule()));
   }
 
   public TestRMWebServicesNodes() {
@@ -227,7 +226,8 @@ public class TestRMWebServicesNodes extends JerseyTestBase {
     assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
     JSONObject json = response.getEntity(JSONObject.class);
     assertEquals("incorrect number of elements", 1, json.length());
-    assertEquals("nodes is not null", JSONObject.NULL, json.get("nodes"));
+    assertEquals("nodes is not empty",
+        new JSONObject().toString(), json.get("nodes").toString());
   }
 
   @Test
@@ -245,7 +245,7 @@ public class TestRMWebServicesNodes extends JerseyTestBase {
     } catch (UniformInterfaceException ue) {
       ClientResponse response = ue.getResponse();
 
-      assertEquals(Status.BAD_REQUEST, response.getClientResponseStatus());
+      assertResponseStatusCode(Status.BAD_REQUEST, response.getStatusInfo());
       assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
 
       JSONObject msg = response.getEntity(JSONObject.class);
@@ -357,7 +357,8 @@ public class TestRMWebServicesNodes extends JerseyTestBase {
     assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
     JSONObject json = response.getEntity(JSONObject.class);
     assertEquals("incorrect number of elements", 1, json.length());
-    assertEquals("nodes is not null", JSONObject.NULL, json.get("nodes"));
+    assertEquals("nodes is not empty",
+        new JSONObject().toString(), json.get("nodes").toString());
   }
 
   public void testNodesHelper(String path, String media) throws JSONException,
@@ -436,7 +437,7 @@ public class TestRMWebServicesNodes extends JerseyTestBase {
       fail("should have thrown exception on non-existent nodeid");
     } catch (UniformInterfaceException ue) {
       ClientResponse response = ue.getResponse();
-      assertEquals(Status.NOT_FOUND, response.getClientResponseStatus());
+      assertResponseStatusCode(Status.NOT_FOUND, response.getStatusInfo());
       assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
       JSONObject msg = response.getEntity(JSONObject.class);
       JSONObject exception = msg.getJSONObject("RemoteException");
@@ -462,7 +463,7 @@ public class TestRMWebServicesNodes extends JerseyTestBase {
       fail("should have thrown exception on non-existent nodeid");
     } catch (UniformInterfaceException ue) {
       ClientResponse response = ue.getResponse();
-      assertEquals(Status.NOT_FOUND, response.getClientResponseStatus());
+      assertResponseStatusCode(Status.NOT_FOUND, response.getStatusInfo());
       assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
       JSONObject msg = response.getEntity(JSONObject.class);
       JSONObject exception = msg.getJSONObject("RemoteException");
@@ -488,7 +489,7 @@ public class TestRMWebServicesNodes extends JerseyTestBase {
       fail("should have thrown exception on non-existent nodeid");
     } catch (UniformInterfaceException ue) {
       ClientResponse response = ue.getResponse();
-      assertEquals(Status.NOT_FOUND, response.getClientResponseStatus());
+      assertResponseStatusCode(Status.NOT_FOUND, response.getStatusInfo());
       assertEquals(MediaType.APPLICATION_XML_TYPE, response.getType());
       String msg = response.getEntity(String.class);
       System.out.println(msg);
@@ -531,7 +532,7 @@ public class TestRMWebServicesNodes extends JerseyTestBase {
     } catch (UniformInterfaceException ue) {
       ClientResponse response = ue.getResponse();
 
-      assertEquals(Status.BAD_REQUEST, response.getClientResponseStatus());
+      assertResponseStatusCode(Status.BAD_REQUEST, response.getStatusInfo());
       assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
       JSONObject msg = response.getEntity(JSONObject.class);
       JSONObject exception = msg.getJSONObject("RemoteException");

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5d58858b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesReservation.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/TestRMWebServicesReservation.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesReservation.java
index 40d46d0..5f55e34 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesReservation.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesReservation.java
@@ -18,6 +18,7 @@
 
 package org.apache.hadoop.yarn.server.resourcemanager.webapp;
 
+import static org.apache.hadoop.yarn.webapp.WebServicesTestUtils.assertResponseStatusCode;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 
@@ -56,6 +57,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.ReservationUpdat
 import org.apache.hadoop.yarn.util.Clock;
 import org.apache.hadoop.yarn.util.UTCClock;
 import org.apache.hadoop.yarn.webapp.GenericExceptionHandler;
+import org.apache.hadoop.yarn.webapp.GuiceServletConfig;
 import org.apache.hadoop.yarn.webapp.JerseyTestBase;
 import org.codehaus.jettison.json.JSONArray;
 import org.codehaus.jettison.json.JSONException;
@@ -70,7 +72,6 @@ import org.junit.runners.Parameterized.Parameters;
 import com.google.inject.Guice;
 import com.google.inject.Injector;
 import com.google.inject.Singleton;
-import com.google.inject.servlet.GuiceServletContextListener;
 import com.google.inject.servlet.ServletModule;
 import com.sun.jersey.api.client.ClientResponse;
 import com.sun.jersey.api.client.ClientResponse.Status;
@@ -86,10 +87,9 @@ import com.sun.jersey.test.framework.WebAppDescriptor;
 public class TestRMWebServicesReservation extends JerseyTestBase {
 
   private String webserviceUserName = "testuser";
-  private boolean setAuthFilter = false;
+  private static boolean setAuthFilter = false;
 
   private static MockRM rm;
-  private static Injector injector;
 
   private static final int MINIMUM_RESOURCE_DURATION = 1000000;
   private static final Clock clock = new UTCClock();
@@ -103,14 +103,6 @@ public class TestRMWebServicesReservation extends JerseyTestBase {
   private static final String GET_NEW_RESERVATION_PATH =
       "reservation/new-reservation";
 
-  public static class GuiceServletConfig extends GuiceServletContextListener {
-
-    @Override
-    protected Injector getInjector() {
-      return injector;
-    }
-  }
-
   /*
    * Helper class to allow testing of RM web services which require
    * authorization Add this class as a filter in the Guice injector for the
@@ -139,7 +131,7 @@ public class TestRMWebServicesReservation extends JerseyTestBase {
 
   }
 
-  private abstract class TestServletModule extends ServletModule {
+  private static abstract class TestServletModule extends ServletModule {
     public Configuration conf = new Configuration();
 
     public abstract void configureScheduler();
@@ -175,7 +167,7 @@ public class TestRMWebServicesReservation extends JerseyTestBase {
     }
   }
 
-  private class CapTestServletModule extends TestServletModule {
+  private static class CapTestServletModule extends TestServletModule {
     @Override
     public void configureScheduler() {
       conf.set("yarn.resourcemanager.scheduler.class",
@@ -183,7 +175,7 @@ public class TestRMWebServicesReservation extends JerseyTestBase {
     }
   }
 
-  private class FairTestServletModule extends TestServletModule {
+  private static class FairTestServletModule extends TestServletModule {
     @Override
     public void configureScheduler() {
       try {
@@ -209,52 +201,65 @@ public class TestRMWebServicesReservation extends JerseyTestBase {
     }
   }
 
-  private Injector getNoAuthInjectorCap() {
-    return Guice.createInjector(new CapTestServletModule() {
-      @Override
-      protected void configureServlets() {
-        setAuthFilter = false;
-        super.configureServlets();
-      }
-    });
+  private static class NoAuthServletModule extends CapTestServletModule {
+    @Override
+    protected void configureServlets() {
+      setAuthFilter = false;
+      super.configureServlets();
+    }
   }
 
-  private Injector getSimpleAuthInjectorCap() {
-    return Guice.createInjector(new CapTestServletModule() {
-      @Override
-      protected void configureServlets() {
-        setAuthFilter = true;
-        conf.setBoolean(YarnConfiguration.YARN_ACL_ENABLE, true);
-        // set the admin acls otherwise all users are considered admins
-        // and we can't test authorization
-        conf.setStrings(YarnConfiguration.YARN_ADMIN_ACL, "testuser1");
-        super.configureServlets();
-      }
-    });
+  private static class SimpleAuthServletModule extends CapTestServletModule {
+    @Override
+    protected void configureServlets() {
+      setAuthFilter = true;
+      conf.setBoolean(YarnConfiguration.YARN_ACL_ENABLE, true);
+      // set the admin acls otherwise all users are considered admins
+      // and we can't test authorization
+      conf.setStrings(YarnConfiguration.YARN_ADMIN_ACL, "testuser1");
+      super.configureServlets();
+    }
   }
 
-  private Injector getNoAuthInjectorFair() {
-    return Guice.createInjector(new FairTestServletModule() {
-      @Override
-      protected void configureServlets() {
-        setAuthFilter = false;
-        super.configureServlets();
-      }
-    });
+  private static class FairNoAuthServletModule extends FairTestServletModule {
+    @Override
+    protected void configureServlets() {
+      setAuthFilter = false;
+      super.configureServlets();
+    }
   }
 
-  private Injector getSimpleAuthInjectorFair() {
-    return Guice.createInjector(new FairTestServletModule() {
-      @Override
-      protected void configureServlets() {
-        setAuthFilter = true;
-        conf.setBoolean(YarnConfiguration.YARN_ACL_ENABLE, true);
-        // set the admin acls otherwise all users are considered admins
-        // and we can't test authorization
-        conf.setStrings(YarnConfiguration.YARN_ADMIN_ACL, "testuser1");
-        super.configureServlets();
-      }
-    });
+  private static class FairSimpleAuthServletModule extends
+      FairTestServletModule {
+    @Override
+    protected void configureServlets() {
+      setAuthFilter = true;
+      conf.setBoolean(YarnConfiguration.YARN_ACL_ENABLE, true);
+      // set the admin acls otherwise all users are considered admins
+      // and we can't test authorization
+      conf.setStrings(YarnConfiguration.YARN_ADMIN_ACL, "testuser1");
+      super.configureServlets();
+    }
+  }
+
+  private Injector initNoAuthInjectorCap() {
+    return GuiceServletConfig.setInjector(
+        Guice.createInjector(new NoAuthServletModule()));
+  }
+
+  private Injector initSimpleAuthInjectorCap() {
+    return GuiceServletConfig.setInjector(
+        Guice.createInjector(new SimpleAuthServletModule()));
+  }
+
+  private Injector initNoAuthInjectorFair() {
+    return GuiceServletConfig.setInjector(
+        Guice.createInjector(new FairNoAuthServletModule()));
+  }
+
+  private Injector initSimpleAuthInjectorFair() {
+    return GuiceServletConfig.setInjector(
+        Guice.createInjector(new FairSimpleAuthServletModule()));
   }
 
   @Parameters
@@ -279,19 +284,19 @@ public class TestRMWebServicesReservation extends JerseyTestBase {
     case 0:
     default:
       // No Auth Capacity Scheduler
-      injector = getNoAuthInjectorCap();
+      initNoAuthInjectorCap();
       break;
     case 1:
       // Simple Auth Capacity Scheduler
-      injector = getSimpleAuthInjectorCap();
+      initSimpleAuthInjectorCap();
       break;
     case 2:
       // No Auth Fair Scheduler
-      injector = getNoAuthInjectorFair();
+      initNoAuthInjectorFair();
       break;
     case 3:
       // Simple Auth Fair Scheduler
-      injector = getSimpleAuthInjectorFair();
+      initSimpleAuthInjectorFair();
       break;
     }
   }
@@ -959,7 +964,7 @@ public class TestRMWebServicesReservation extends JerseyTestBase {
         .post(ClientResponse.class);
 
     if (!this.isAuthenticationEnabled()) {
-      assertEquals(Status.UNAUTHORIZED, response.getClientResponseStatus());
+      assertResponseStatusCode(Status.UNAUTHORIZED, response.getStatusInfo());
       return ReservationId.newInstance(clock.getTime(), fallbackReservationId);
     }
 
@@ -1013,7 +1018,7 @@ public class TestRMWebServicesReservation extends JerseyTestBase {
             .accept(media).post(ClientResponse.class);
 
     if (!this.isAuthenticationEnabled()) {
-      assertEquals(Status.UNAUTHORIZED, response.getClientResponseStatus());
+      assertResponseStatusCode(Status.UNAUTHORIZED, response.getStatusInfo());
     }
 
     return response;
@@ -1046,13 +1051,13 @@ public class TestRMWebServicesReservation extends JerseyTestBase {
             .accept(media).post(ClientResponse.class);
 
     if (!this.isAuthenticationEnabled()) {
-      assertEquals(Status.UNAUTHORIZED, response.getClientResponseStatus());
+      assertResponseStatusCode(Status.UNAUTHORIZED, response.getStatusInfo());
       return;
     }
 
     System.out.println("RESPONSE:" + response);
     assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
-    assertEquals(Status.OK, response.getClientResponseStatus());
+    assertResponseStatusCode(Status.OK, response.getStatusInfo());
 
   }
 
@@ -1095,13 +1100,13 @@ public class TestRMWebServicesReservation extends JerseyTestBase {
             .accept(media).post(ClientResponse.class);
 
     if (!this.isAuthenticationEnabled()) {
-      assertEquals(Status.UNAUTHORIZED, response.getClientResponseStatus());
+      assertResponseStatusCode(Status.UNAUTHORIZED, response.getStatusInfo());
       return;
     }
 
     System.out.println("RESPONSE:" + response);
     assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
-    assertEquals(Status.OK, response.getClientResponseStatus());
+    assertResponseStatusCode(Status.OK, response.getStatusInfo());
   }
 
   private void testRDLHelper(JSONObject json) throws JSONException {
@@ -1126,12 +1131,12 @@ public class TestRMWebServicesReservation extends JerseyTestBase {
     ClientResponse response = resource.get(ClientResponse.class);
 
     if (!this.isAuthenticationEnabled()) {
-      assertEquals(Status.UNAUTHORIZED, response.getClientResponseStatus());
+      assertResponseStatusCode(Status.UNAUTHORIZED, response.getStatusInfo());
       return null;
     }
 
     assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
-    assertEquals(status, response.getClientResponseStatus());
+    assertResponseStatusCode(status, response.getStatusInfo());
 
     return response.getEntity(JSONObject.class);
   }
@@ -1152,7 +1157,7 @@ public class TestRMWebServicesReservation extends JerseyTestBase {
   }
 
   private boolean isHttpSuccessResponse(ClientResponse response) {
-    return (response.getStatus() / 100) == 2;
+    return (response.getStatusInfo().getStatusCode() / 100) == 2;
   }
 
   private void setupCluster(int nodes) throws Exception {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5d58858b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/webapp/TestRMWithCSRFFilter.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/webapp/TestRMWithCSRFFilter.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/webapp/TestRMWithCSRFFilter.java
index 2efbd2d..a15d656 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/webapp/TestRMWithCSRFFilter.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/webapp/TestRMWithCSRFFilter.java
@@ -19,12 +19,9 @@
 package org.apache.hadoop.yarn.webapp;
 
 import com.google.inject.Guice;
-import com.google.inject.Injector;
-import com.google.inject.servlet.GuiceServletContextListener;
 import com.google.inject.servlet.ServletModule;
 import com.sun.jersey.api.client.ClientResponse;
 import com.sun.jersey.api.client.ClientResponse.Status;
-import com.sun.jersey.api.client.UniformInterfaceException;
 import com.sun.jersey.api.client.WebResource;
 import com.sun.jersey.guice.spi.container.servlet.GuiceContainer;
 import com.sun.jersey.test.framework.WebAppDescriptor;
@@ -33,7 +30,6 @@ import org.apache.hadoop.security.http.RestCsrfPreventionFilter;
 import org.apache.hadoop.service.Service.STATE;
 import org.apache.hadoop.util.VersionInfo;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
-import org.apache.hadoop.yarn.server.resourcemanager.ClusterMetrics;
 import org.apache.hadoop.yarn.server.resourcemanager.MockRM;
 import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
@@ -41,10 +37,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fifo.FifoSchedule
 import org.apache.hadoop.yarn.server.resourcemanager.webapp.JAXBContextResolver;
 import org.apache.hadoop.yarn.server.resourcemanager.webapp.RMWebServices;
 import org.apache.hadoop.yarn.util.YarnVersionInfo;
-import org.codehaus.jettison.json.JSONException;
-import org.codehaus.jettison.json.JSONObject;
 import org.junit.Before;
-import org.junit.BeforeClass;
 import org.junit.Test;
 import org.w3c.dom.Document;
 import org.w3c.dom.Element;
@@ -60,7 +53,6 @@ import java.util.Map;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
 
 /**
  * Used TestRMWebServices as an example of web invocations of RM and added
@@ -70,7 +62,7 @@ public class TestRMWithCSRFFilter extends JerseyTestBase {
 
   private static MockRM rm;
 
-  private Injector injector = Guice.createInjector(new ServletModule() {
+  private static class WebServletModule extends ServletModule {
     @Override
     protected void configureServlets() {
       bind(JAXBContextResolver.class);
@@ -89,20 +81,14 @@ public class TestRMWithCSRFFilter extends JerseyTestBase {
                      "OPTIONS,HEAD,TRACE");
       filter("/*").through(csrfFilter, initParams);
     }
-  });
-
-  public class GuiceServletConfig extends GuiceServletContextListener {
-
-    @Override
-    protected Injector getInjector() {
-      return injector;
-    }
-  }
+  };
 
   @Before
   @Override
   public void setUp() throws Exception {
     super.setUp();
+    GuiceServletConfig.setInjector(
+        Guice.createInjector(new WebServletModule()));
   }
 
   public TestRMWithCSRFFilter() {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5d58858b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/webapp/TestRMWithXFSFilter.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/webapp/TestRMWithXFSFilter.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/webapp/TestRMWithXFSFilter.java
index a73a0ed..bcf2b21 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/webapp/TestRMWithXFSFilter.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/webapp/TestRMWithXFSFilter.java
@@ -19,8 +19,6 @@
 package org.apache.hadoop.yarn.webapp;
 
 import com.google.inject.Guice;
-import com.google.inject.Injector;
-import com.google.inject.servlet.GuiceServletContextListener;
 import com.google.inject.servlet.ServletModule;
 import com.sun.jersey.api.client.ClientResponse;
 import com.sun.jersey.api.client.WebResource;
@@ -52,19 +50,6 @@ public class TestRMWithXFSFilter extends JerseyTestBase {
 
   private static MockRM rm;
 
-  private Injector injector;
-
-  /**
-   *
-   */
-  public class GuiceServletConfig extends GuiceServletContextListener {
-
-    @Override
-    protected Injector getInjector() {
-      return injector;
-    }
-  }
-
   @Before
   @Override
   public void setUp() throws Exception {
@@ -103,7 +88,7 @@ public class TestRMWithXFSFilter extends JerseyTestBase {
 
   protected void createInjector(final String headerValue,
       final boolean explicitlyDisabled) {
-    injector = Guice.createInjector(new ServletModule() {
+    GuiceServletConfig.setInjector(Guice.createInjector(new ServletModule() {
       @Override
       protected void configureServlets() {
         bind(JAXBContextResolver.class);
@@ -127,7 +112,7 @@ public class TestRMWithXFSFilter extends JerseyTestBase {
 
         filter("/*").through(xfsFilter, initParams);
       }
-    });
+    }));
   }
 
   @Test


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


[10/17] hadoop git commit: HDFS-10538. Remove AsyncDistributedFileSystem. Contributed by Xiaobing Zhou and Jitendra Pandey.

Posted by ae...@apache.org.
HDFS-10538. Remove AsyncDistributedFileSystem. Contributed by Xiaobing Zhou and Jitendra Pandey.


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

Branch: refs/heads/HDFS-1312
Commit: 7adc4d17691816ad32d8d71974a62b9f920cb4c2
Parents: b7c4cf7
Author: Jitendra Pandey <ji...@apache.org>
Authored: Mon Jun 20 17:22:55 2016 -0700
Committer: Jitendra Pandey <ji...@apache.org>
Committed: Mon Jun 20 17:22:55 2016 -0700

----------------------------------------------------------------------
 .../hadoop/hdfs/AsyncDistributedFileSystem.java | 213 ---------
 .../hadoop/hdfs/DistributedFileSystem.java      |  10 -
 .../org/apache/hadoop/hdfs/TestAsyncDFS.java    | 454 -------------------
 .../apache/hadoop/hdfs/TestAsyncDFSRename.java  | 187 --------
 .../apache/hadoop/hdfs/TestAsyncHDFSWithHA.java | 181 --------
 5 files changed, 1045 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/7adc4d17/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/AsyncDistributedFileSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/AsyncDistributedFileSystem.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/AsyncDistributedFileSystem.java
deleted file mode 100644
index 824336a..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/AsyncDistributedFileSystem.java
+++ /dev/null
@@ -1,213 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hdfs;
-
-import java.io.IOException;
-import java.util.List;
-import java.util.concurrent.Future;
-
-import org.apache.hadoop.classification.InterfaceStability.Unstable;
-import org.apache.hadoop.fs.Options;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.permission.AclEntry;
-import org.apache.hadoop.fs.permission.AclStatus;
-import org.apache.hadoop.fs.permission.FsPermission;
-import org.apache.hadoop.hdfs.DFSOpsCountStatistics.OpType;
-import org.apache.hadoop.io.retry.AsyncCallHandler;
-import org.apache.hadoop.util.concurrent.AsyncGetFuture;
-import org.apache.hadoop.ipc.Client;
-
-/****************************************************************
- * Implementation of the asynchronous distributed file system.
- * This instance of this class is the way end-user code interacts
- * with a Hadoop DistributedFileSystem in an asynchronous manner.
- *
- * This class is unstable, so no guarantee is provided as to reliability,
- * stability or compatibility across any level of release granularity.
- *
- *****************************************************************/
-@Unstable
-public class AsyncDistributedFileSystem {
-
-  private final DistributedFileSystem dfs;
-
-  AsyncDistributedFileSystem(final DistributedFileSystem dfs) {
-    this.dfs = dfs;
-  }
-
-  private static <T> Future<T> getReturnValue() {
-    return new AsyncGetFuture<>(AsyncCallHandler.getAsyncReturn());
-  }
-
-  /**
-   * Renames Path src to Path dst
-   * <ul>
-   * <li>Fails if src is a file and dst is a directory.
-   * <li>Fails if src is a directory and dst is a file.
-   * <li>Fails if the parent of dst does not exist or is a file.
-   * </ul>
-   * <p>
-   * If OVERWRITE option is not passed as an argument, rename fails if the dst
-   * already exists.
-   * <p>
-   * If OVERWRITE option is passed as an argument, rename overwrites the dst if
-   * it is a file or an empty directory. Rename fails if dst is a non-empty
-   * directory.
-   * <p>
-   * Note that atomicity of rename is dependent on the file system
-   * implementation. Please refer to the file system documentation for details.
-   * This default implementation is non atomic.
-   *
-   * @param src
-   *          path to be renamed
-   * @param dst
-   *          new path after rename
-   * @throws IOException
-   *           on failure
-   * @return an instance of Future, #get of which is invoked to wait for
-   *         asynchronous call being finished.
-   */
-  public Future<Void> rename(Path src, Path dst,
-      final Options.Rename... options) throws IOException {
-    dfs.getFsStatistics().incrementWriteOps(1);
-    dfs.getDFSOpsCountStatistics().incrementOpCounter(OpType.RENAME);
-
-    final Path absSrc = dfs.fixRelativePart(src);
-    final Path absDst = dfs.fixRelativePart(dst);
-
-    final boolean isAsync = Client.isAsynchronousMode();
-    Client.setAsynchronousMode(true);
-    try {
-      dfs.getClient().rename(dfs.getPathName(absSrc), dfs.getPathName(absDst),
-          options);
-      return getReturnValue();
-    } finally {
-      Client.setAsynchronousMode(isAsync);
-    }
-  }
-
-  /**
-   * Set permission of a path.
-   *
-   * @param p
-   *          the path the permission is set to
-   * @param permission
-   *          the permission that is set to a path.
-   * @return an instance of Future, #get of which is invoked to wait for
-   *         asynchronous call being finished.
-   */
-  public Future<Void> setPermission(Path p, final FsPermission permission)
-      throws IOException {
-    dfs.getFsStatistics().incrementWriteOps(1);
-    dfs.getDFSOpsCountStatistics().incrementOpCounter(OpType.SET_PERMISSION);
-    final Path absPath = dfs.fixRelativePart(p);
-    final boolean isAsync = Client.isAsynchronousMode();
-    Client.setAsynchronousMode(true);
-    try {
-      dfs.getClient().setPermission(dfs.getPathName(absPath), permission);
-      return getReturnValue();
-    } finally {
-      Client.setAsynchronousMode(isAsync);
-    }
-  }
-
-  /**
-   * Set owner of a path (i.e. a file or a directory). The parameters username
-   * and groupname cannot both be null.
-   *
-   * @param p
-   *          The path
-   * @param username
-   *          If it is null, the original username remains unchanged.
-   * @param groupname
-   *          If it is null, the original groupname remains unchanged.
-   * @return an instance of Future, #get of which is invoked to wait for
-   *         asynchronous call being finished.
-   */
-  public Future<Void> setOwner(Path p, String username, String groupname)
-      throws IOException {
-    if (username == null && groupname == null) {
-      throw new IOException("username == null && groupname == null");
-    }
-
-    dfs.getFsStatistics().incrementWriteOps(1);
-    dfs.getDFSOpsCountStatistics().incrementOpCounter(OpType.SET_OWNER);
-    final Path absPath = dfs.fixRelativePart(p);
-    final boolean isAsync = Client.isAsynchronousMode();
-    Client.setAsynchronousMode(true);
-    try {
-      dfs.getClient().setOwner(dfs.getPathName(absPath), username, groupname);
-      return getReturnValue();
-    } finally {
-      Client.setAsynchronousMode(isAsync);
-    }
-  }
-
-  /**
-   * Fully replaces ACL of files and directories, discarding all existing
-   * entries.
-   *
-   * @param p
-   *          Path to modify
-   * @param aclSpec
-   *          List<AclEntry> describing modifications, must include entries for
-   *          user, group, and others for compatibility with permission bits.
-   * @throws IOException
-   *           if an ACL could not be modified
-   * @return an instance of Future, #get of which is invoked to wait for
-   *         asynchronous call being finished.
-   */
-  public Future<Void> setAcl(Path p, final List<AclEntry> aclSpec)
-      throws IOException {
-    dfs.getFsStatistics().incrementWriteOps(1);
-    dfs.getDFSOpsCountStatistics().incrementOpCounter(OpType.SET_ACL);
-    final Path absPath = dfs.fixRelativePart(p);
-    final boolean isAsync = Client.isAsynchronousMode();
-    Client.setAsynchronousMode(true);
-    try {
-      dfs.getClient().setAcl(dfs.getPathName(absPath), aclSpec);
-      return getReturnValue();
-    } finally {
-      Client.setAsynchronousMode(isAsync);
-    }
-  }
-
-  /**
-   * Gets the ACL of a file or directory.
-   *
-   * @param p
-   *          Path to get
-   * @return AclStatus describing the ACL of the file or directory
-   * @throws IOException
-   *           if an ACL could not be read
-   * @return an instance of Future, #get of which is invoked to wait for
-   *         asynchronous call being finished.
-   */
-  public Future<AclStatus> getAclStatus(Path p) throws IOException {
-    final Path absPath = dfs.fixRelativePart(p);
-    final boolean isAsync = Client.isAsynchronousMode();
-    Client.setAsynchronousMode(true);
-    try {
-      dfs.getClient().getAclStatus(dfs.getPathName(absPath));
-      return getReturnValue();
-    } finally {
-      Client.setAsynchronousMode(isAsync);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7adc4d17/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java
index 66ee42f..24ffb40 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java
@@ -31,7 +31,6 @@ import java.util.Map;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.BlockLocation;
 import org.apache.hadoop.fs.BlockStoragePolicySpi;
@@ -2511,15 +2510,6 @@ public class DistributedFileSystem extends FileSystem {
     return ret;
   }
 
-  private final AsyncDistributedFileSystem adfs =
-      new AsyncDistributedFileSystem(this);
-
-  /** @return an {@link AsyncDistributedFileSystem} object. */
-  @Unstable
-  public AsyncDistributedFileSystem getAsyncDistributedFileSystem() {
-    return adfs;
-  }
-
   @Override
   protected Path fixRelativePart(Path p) {
     return super.fixRelativePart(p);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7adc4d17/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestAsyncDFS.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestAsyncDFS.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestAsyncDFS.java
deleted file mode 100644
index 6a60290..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestAsyncDFS.java
+++ /dev/null
@@ -1,454 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hdfs;
-
-import static org.apache.hadoop.fs.permission.AclEntryScope.ACCESS;
-import static org.apache.hadoop.fs.permission.AclEntryScope.DEFAULT;
-import static org.apache.hadoop.fs.permission.AclEntryType.GROUP;
-import static org.apache.hadoop.fs.permission.AclEntryType.MASK;
-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.NONE;
-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.assertArrayEquals;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.fail;
-
-import java.io.IOException;
-import java.security.PrivilegedExceptionAction;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Random;
-import java.util.concurrent.ExecutionException;
-import java.util.concurrent.Future;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.CommonConfigurationKeys;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.Options.Rename;
-import org.apache.hadoop.fs.permission.AclEntry;
-import org.apache.hadoop.fs.permission.AclStatus;
-import org.apache.hadoop.fs.permission.FsPermission;
-import org.apache.hadoop.hdfs.TestDFSPermission.PermissionGenerator;
-import org.apache.hadoop.hdfs.server.namenode.AclTestHelpers;
-import org.apache.hadoop.hdfs.server.namenode.FSAclBaseTest;
-import org.apache.hadoop.ipc.AsyncCallLimitExceededException;
-import org.apache.hadoop.ipc.RemoteException;
-import org.apache.hadoop.security.UserGroupInformation;
-import org.apache.hadoop.util.Time;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Test;
-
-import com.google.common.collect.Lists;
-
-/**
- * Unit tests for asynchronous distributed filesystem.
- * */
-public class TestAsyncDFS {
-  public static final Log LOG = LogFactory.getLog(TestAsyncDFS.class);
-  private final short replFactor = 1;
-  private final long blockSize = 512;
-  private long fileLen = 0;
-  private final long seed = Time.now();
-  private final Random r = new Random(seed);
-  private final PermissionGenerator permGenerator = new PermissionGenerator(r);
-  private static final int NUM_TESTS = 50;
-  private static final int NUM_NN_HANDLER = 10;
-  private static final int ASYNC_CALL_LIMIT = 1000;
-
-  private Configuration conf;
-  private MiniDFSCluster cluster;
-  private DistributedFileSystem fs;
-  private AsyncDistributedFileSystem adfs;
-
-  @Before
-  public void setup() throws IOException {
-    conf = new HdfsConfiguration();
-    // explicitly turn on acl
-    conf.setBoolean(DFSConfigKeys.DFS_NAMENODE_ACLS_ENABLED_KEY, true);
-    // explicitly turn on permission checking
-    conf.setBoolean(DFSConfigKeys.DFS_PERMISSIONS_ENABLED_KEY, true);
-    // set the limit of max async calls
-    conf.setInt(CommonConfigurationKeys.IPC_CLIENT_ASYNC_CALLS_MAX_KEY,
-        ASYNC_CALL_LIMIT);
-    // set server handlers
-    conf.setInt(DFSConfigKeys.DFS_NAMENODE_HANDLER_COUNT_KEY, NUM_NN_HANDLER);
-    cluster = new MiniDFSCluster.Builder(conf).numDataNodes(0).build();
-    cluster.waitActive();
-    fs = cluster.getFileSystem();
-    adfs = fs.getAsyncDistributedFileSystem();
-  }
-
-  @After
-  public void tearDown() throws IOException {
-    if (fs != null) {
-      fs.close();
-      fs = null;
-    }
-    if (cluster != null) {
-      cluster.shutdown();
-      cluster = null;
-    }
-  }
-
-  @Test(timeout=60000)
-  public void testBatchAsyncAcl() throws Exception {
-    final String basePath = "testBatchAsyncAcl";
-    final Path parent = new Path(String.format("/test/%s/", basePath));
-
-    // prepare test
-    final Path[] paths = new Path[NUM_TESTS];
-    for (int i = 0; i < NUM_TESTS; i++) {
-      paths[i] = new Path(parent, "acl" + i);
-      FileSystem.mkdirs(fs, paths[i],
-          FsPermission.createImmutable((short) 0750));
-      assertTrue(fs.exists(paths[i]));
-      assertTrue(fs.getFileStatus(paths[i]).isDirectory());
-    }
-
-    final List<AclEntry> aclSpec = getAclSpec();
-    final AclEntry[] expectedAclSpec = getExpectedAclSpec();
-    Map<Integer, Future<Void>> setAclRetFutures =
-        new HashMap<Integer, Future<Void>>();
-    Map<Integer, Future<AclStatus>> getAclRetFutures =
-        new HashMap<Integer, Future<AclStatus>>();
-    int start = 0, end = 0;
-    try {
-      // test setAcl
-      for (int i = 0; i < NUM_TESTS; i++) {
-        for (;;) {
-          try {
-            Future<Void> retFuture = adfs.setAcl(paths[i], aclSpec);
-            setAclRetFutures.put(i, retFuture);
-            break;
-          } catch (AsyncCallLimitExceededException e) {
-            start = end;
-            end = i;
-            waitForAclReturnValues(setAclRetFutures, start, end);
-          }
-        }
-      }
-      waitForAclReturnValues(setAclRetFutures, end, NUM_TESTS);
-
-      // test getAclStatus
-      start = 0;
-      end = 0;
-      for (int i = 0; i < NUM_TESTS; i++) {
-        for (;;) {
-          try {
-            Future<AclStatus> retFuture = adfs.getAclStatus(paths[i]);
-            getAclRetFutures.put(i, retFuture);
-            break;
-          } catch (AsyncCallLimitExceededException e) {
-            start = end;
-            end = i;
-            waitForAclReturnValues(getAclRetFutures, start, end, paths,
-                expectedAclSpec);
-          }
-        }
-      }
-      waitForAclReturnValues(getAclRetFutures, end, NUM_TESTS, paths,
-          expectedAclSpec);
-    } catch (Exception e) {
-      throw e;
-    }
-  }
-
-  static void waitForReturnValues(final Map<Integer, Future<Void>> retFutures,
-      final int start, final int end)
-      throws InterruptedException, ExecutionException {
-    LOG.info(String.format("calling waitForReturnValues [%d, %d)", start, end));
-    for (int i = start; i < end; i++) {
-      LOG.info("calling Future#get #" + i);
-      retFutures.get(i).get();
-    }
-  }
-
-  private void waitForAclReturnValues(
-      final Map<Integer, Future<Void>> aclRetFutures, final int start,
-      final int end) throws InterruptedException, ExecutionException {
-    for (int i = start; i < end; i++) {
-      aclRetFutures.get(i).get();
-    }
-  }
-
-  private void waitForAclReturnValues(
-      final Map<Integer, Future<AclStatus>> aclRetFutures, final int start,
-      final int end, final Path[] paths, final AclEntry[] expectedAclSpec)
-      throws InterruptedException, ExecutionException, IOException {
-    for (int i = start; i < end; i++) {
-      AclStatus aclStatus = aclRetFutures.get(i).get();
-      verifyGetAcl(aclStatus, expectedAclSpec, paths[i]);
-    }
-  }
-
-  private void verifyGetAcl(final AclStatus aclStatus,
-      final AclEntry[] expectedAclSpec, final Path path) throws IOException {
-    if (aclStatus == null) {
-      return;
-    }
-
-    // verify permission and acl
-    AclEntry[] returned = aclStatus.getEntries().toArray(new AclEntry[0]);
-    assertArrayEquals(expectedAclSpec, returned);
-    assertPermission(path, (short) 010770);
-    FSAclBaseTest.assertAclFeature(cluster, path, true);
-  }
-
-  private List<AclEntry> getAclSpec() {
-    return Lists.newArrayList(
-        aclEntry(ACCESS, USER, ALL),
-        aclEntry(ACCESS, USER, "foo", ALL),
-        aclEntry(ACCESS, GROUP, READ_EXECUTE),
-        aclEntry(ACCESS, OTHER, NONE),
-        aclEntry(DEFAULT, USER, "foo", ALL));
-  }
-
-  private AclEntry[] getExpectedAclSpec() {
-    return new AclEntry[] {
-        aclEntry(ACCESS, USER, "foo", ALL),
-        aclEntry(ACCESS, GROUP, READ_EXECUTE),
-        aclEntry(DEFAULT, USER, ALL),
-        aclEntry(DEFAULT, USER, "foo", ALL),
-        aclEntry(DEFAULT, GROUP, READ_EXECUTE),
-        aclEntry(DEFAULT, MASK, ALL),
-        aclEntry(DEFAULT, OTHER, NONE) };
-  }
-
-  private void assertPermission(final Path pathToCheck, final short perm)
-      throws IOException {
-    AclTestHelpers.assertPermission(fs, pathToCheck, perm);
-  }
-
-  @Test(timeout=60000)
-  public void testAsyncAPIWithException() throws Exception {
-    String group1 = "group1";
-    String group2 = "group2";
-    String user1 = "user1";
-    UserGroupInformation ugi1;
-
-    // create fake mapping for the groups
-    Map<String, String[]> u2gMap = new HashMap<String, String[]>(1);
-    u2gMap.put(user1, new String[] {group1, group2});
-    DFSTestUtil.updateConfWithFakeGroupMapping(conf, u2gMap);
-
-    // Initiate all four users
-    ugi1 = UserGroupInformation.createUserForTesting(user1, new String[] {
-        group1, group2 });
-
-    final Path parent = new Path("/test/async_api_exception/");
-    final Path aclDir = new Path(parent, "aclDir");
-    final Path src = new Path(parent, "src");
-    final Path dst = new Path(parent, "dst");
-    fs.mkdirs(aclDir, FsPermission.createImmutable((short) 0700));
-    fs.mkdirs(src);
-
-    AsyncDistributedFileSystem adfs1 = ugi1
-        .doAs(new PrivilegedExceptionAction<AsyncDistributedFileSystem>() {
-          @Override
-          public AsyncDistributedFileSystem run() throws Exception {
-            return cluster.getFileSystem().getAsyncDistributedFileSystem();
-          }
-        });
-
-    Future<Void> retFuture;
-    // test rename
-    try {
-      retFuture = adfs1.rename(src, dst, Rename.OVERWRITE);
-      retFuture.get();
-    } catch (ExecutionException e) {
-      checkPermissionDenied(e, src, user1);
-      assertTrue("Permission denied messages must carry the path parent", e
-          .getMessage().contains(src.getParent().toUri().getPath()));
-    }
-
-    // test setPermission
-    FsPermission fsPerm = new FsPermission(permGenerator.next());
-    try {
-      retFuture = adfs1.setPermission(src, fsPerm);
-      retFuture.get();
-    } catch (ExecutionException e) {
-      checkPermissionDenied(e, src, user1);
-    }
-
-    // test setOwner
-    try {
-      retFuture = adfs1.setOwner(src, "user1", "group2");
-      retFuture.get();
-    } catch (ExecutionException e) {
-      checkPermissionDenied(e, src, user1);
-    }
-
-    // test setAcl
-    try {
-      retFuture = adfs1.setAcl(aclDir,
-          Lists.newArrayList(aclEntry(ACCESS, USER, ALL)));
-      retFuture.get();
-      fail("setAcl should fail with permission denied");
-    } catch (ExecutionException e) {
-      checkPermissionDenied(e, aclDir, user1);
-    }
-
-    // test getAclStatus
-    try {
-      Future<AclStatus> aclRetFuture = adfs1.getAclStatus(aclDir);
-      aclRetFuture.get();
-      fail("getAclStatus should fail with permission denied");
-    } catch (ExecutionException e) {
-      checkPermissionDenied(e, aclDir, user1);
-    }
-  }
-
-  public static void checkPermissionDenied(final Exception e, final Path dir,
-      final String user) {
-    assertTrue(e.getCause() instanceof RemoteException);
-    assertTrue("Permission denied messages must carry AccessControlException",
-        e.getMessage().contains("AccessControlException"));
-    assertTrue("Permission denied messages must carry the username", e
-        .getMessage().contains(user));
-    assertTrue("Permission denied messages must carry the name of the path",
-        e.getMessage().contains(dir.getName()));
-  }
-
-
-  @Test(timeout = 120000)
-  public void testConcurrentAsyncAPI() throws Exception {
-    String group1 = "group1";
-    String group2 = "group2";
-    String user1 = "user1";
-
-    // create fake mapping for the groups
-    Map<String, String[]> u2gMap = new HashMap<String, String[]>(1);
-    u2gMap.put(user1, new String[] {group1, group2});
-    DFSTestUtil.updateConfWithFakeGroupMapping(conf, u2gMap);
-
-    // prepare for test
-    final Path parent = new Path(
-        String.format("/test/%s/", "testConcurrentAsyncAPI"));
-    final Path[] srcs = new Path[NUM_TESTS];
-    final Path[] dsts = new Path[NUM_TESTS];
-    short[] permissions = new short[NUM_TESTS];
-    for (int i = 0; i < NUM_TESTS; i++) {
-      srcs[i] = new Path(parent, "src" + i);
-      dsts[i] = new Path(parent, "dst" + i);
-      DFSTestUtil.createFile(fs, srcs[i], fileLen, replFactor, 1);
-      DFSTestUtil.createFile(fs, dsts[i], fileLen, replFactor, 1);
-      assertTrue(fs.exists(srcs[i]));
-      assertTrue(fs.getFileStatus(srcs[i]).isFile());
-      assertTrue(fs.exists(dsts[i]));
-      assertTrue(fs.getFileStatus(dsts[i]).isFile());
-      permissions[i] = permGenerator.next();
-    }
-
-    Map<Integer, Future<Void>> renameRetFutures =
-        new HashMap<Integer, Future<Void>>();
-    Map<Integer, Future<Void>> permRetFutures =
-        new HashMap<Integer, Future<Void>>();
-    Map<Integer, Future<Void>> ownerRetFutures =
-        new HashMap<Integer, Future<Void>>();
-    int start = 0, end = 0;
-    // test rename
-    for (int i = 0; i < NUM_TESTS; i++) {
-      for (;;) {
-        try {
-          Future<Void> returnFuture = adfs.rename(srcs[i], dsts[i],
-              Rename.OVERWRITE);
-          renameRetFutures.put(i, returnFuture);
-          break;
-        } catch (AsyncCallLimitExceededException e) {
-          start = end;
-          end = i;
-          waitForReturnValues(renameRetFutures, start, end);
-        }
-      }
-    }
-
-    // wait for completing the calls
-    waitForAclReturnValues(renameRetFutures, end, NUM_TESTS);
-
-    // verify the src should not exist, dst should
-    for (int i = 0; i < NUM_TESTS; i++) {
-      assertFalse(fs.exists(srcs[i]));
-      assertTrue(fs.exists(dsts[i]));
-    }
-
-    // test permissions
-    for (int i = 0; i < NUM_TESTS; i++) {
-      for (;;) {
-        try {
-          Future<Void> retFuture = adfs.setPermission(dsts[i],
-              new FsPermission(permissions[i]));
-          permRetFutures.put(i, retFuture);
-          break;
-        } catch (AsyncCallLimitExceededException e) {
-          start = end;
-          end = i;
-          waitForReturnValues(permRetFutures, start, end);
-        }
-      }
-    }
-    // wait for completing the calls
-    waitForAclReturnValues(permRetFutures, end, NUM_TESTS);
-
-    // verify the permission
-    for (int i = 0; i < NUM_TESTS; i++) {
-      assertTrue(fs.exists(dsts[i]));
-      FsPermission fsPerm = new FsPermission(permissions[i]);
-      fs.access(dsts[i], fsPerm.getUserAction());
-    }
-
-    // test setOwner
-    start = 0;
-    end = 0;
-    for (int i = 0; i < NUM_TESTS; i++) {
-      for (;;) {
-        try {
-          Future<Void> retFuture = adfs.setOwner(dsts[i], "user1", "group2");
-          ownerRetFutures.put(i, retFuture);
-          break;
-        } catch (AsyncCallLimitExceededException e) {
-          start = end;
-          end = i;
-          waitForReturnValues(ownerRetFutures, start, end);
-        }
-      }
-    }
-    // wait for completing the calls
-    waitForAclReturnValues(ownerRetFutures, end, NUM_TESTS);
-
-    // verify the owner
-    for (int i = 0; i < NUM_TESTS; i++) {
-      assertTrue(fs.exists(dsts[i]));
-      assertTrue("user1".equals(fs.getFileStatus(dsts[i]).getOwner()));
-      assertTrue("group2".equals(fs.getFileStatus(dsts[i]).getGroup()));
-    }
-  }
-
-  @Test
-  public void testAsyncWithoutRetry() throws Exception {
-    TestAsyncHDFSWithHA.runTestAsyncWithoutRetry(conf, cluster, fs);
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7adc4d17/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestAsyncDFSRename.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestAsyncDFSRename.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestAsyncDFSRename.java
deleted file mode 100644
index 8d3e509..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestAsyncDFSRename.java
+++ /dev/null
@@ -1,187 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hdfs;
-
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.assertEquals;
-
-import java.io.IOException;
-import java.util.HashMap;
-import java.util.Map;
-import java.util.concurrent.ExecutionException;
-import java.util.concurrent.Future;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.CommonConfigurationKeys;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.Options.Rename;
-import org.apache.hadoop.ipc.AsyncCallLimitExceededException;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Test;
-
-public class TestAsyncDFSRename {
-  public static final Log LOG = LogFactory.getLog(TestAsyncDFSRename.class);
-  private final short replFactor = 1;
-  private final long blockSize = 512;
-  private long fileLen = blockSize * 3;
-  private static final int NUM_TESTS = 50;
-  private static final int NUM_NN_HANDLER = 10;
-  private static final int ASYNC_CALL_LIMIT = 1000;
-
-  private Configuration conf;
-  private MiniDFSCluster cluster;
-  private FileSystem fs;
-  private AsyncDistributedFileSystem adfs;
-
-  @Before
-  public void setup() throws IOException {
-    conf = new HdfsConfiguration();
-    // set the limit of max async calls
-    conf.setInt(CommonConfigurationKeys.IPC_CLIENT_ASYNC_CALLS_MAX_KEY,
-        ASYNC_CALL_LIMIT);
-    // set server handlers
-    conf.setInt(DFSConfigKeys.DFS_NAMENODE_HANDLER_COUNT_KEY, NUM_NN_HANDLER);
-    cluster = new MiniDFSCluster.Builder(conf).numDataNodes(1).build();
-    cluster.waitActive();
-    fs = FileSystem.get(conf);
-    adfs = cluster.getFileSystem().getAsyncDistributedFileSystem();
-  }
-
-  @After
-  public void tearDown() throws IOException {
-    if (fs != null) {
-      fs.close();
-      fs = null;
-    }
-    if (cluster != null) {
-      cluster.shutdown();
-      cluster = null;
-    }
-  }
-
-  @Test(timeout = 60000)
-  public void testCallGetReturnValueMultipleTimes() throws Exception {
-    final Path parent = new Path("/test/testCallGetReturnValueMultipleTimes/");
-    assertTrue(fs.mkdirs(parent));
-
-    // prepare test
-    final Path[] srcs = new Path[NUM_TESTS];
-    final Path[] dsts = new Path[NUM_TESTS];
-    for (int i = 0; i < NUM_TESTS; i++) {
-      srcs[i] = new Path(parent, "src" + i);
-      dsts[i] = new Path(parent, "dst" + i);
-      DFSTestUtil.createFile(fs, srcs[i], fileLen, replFactor, 1);
-      DFSTestUtil.createFile(fs, dsts[i], fileLen, replFactor, 1);
-    }
-
-    // concurrently invoking many rename
-    final Map<Integer, Future<Void>> reFutures =
-        new HashMap<Integer, Future<Void>>();
-    for (int i = 0; i < NUM_TESTS; i++) {
-      Future<Void> retFuture = adfs.rename(srcs[i], dsts[i],
-          Rename.OVERWRITE);
-      reFutures.put(i, retFuture);
-    }
-
-    assertEquals(NUM_TESTS, reFutures.size());
-
-    for (int i = 0; i < 5; i++) {
-      verifyCallGetReturnValueMultipleTimes(reFutures, srcs, dsts);
-    }
-  }
-
-  private void verifyCallGetReturnValueMultipleTimes(
-      final Map<Integer, Future<Void>> reFutures, final Path[] srcs,
-      final Path[] dsts)
-      throws InterruptedException, ExecutionException, IOException {
-
-    // wait for completing the calls
-    waitForReturnValues(reFutures, 0, NUM_TESTS);
-
-    // verify the src dir should not exist, dst should
-    verifyRenames(srcs, dsts);
-  }
-
-  @Test(timeout = 60000)
-  public void testConcurrentAsyncRename() throws Exception {
-    final Path parent = new Path(
-        String.format("/test/%s/", "testConcurrentAsyncRename"));
-    assertTrue(fs.mkdirs(parent));
-
-    // prepare test
-    final Path[] srcs = new Path[NUM_TESTS];
-    final Path[] dsts = new Path[NUM_TESTS];
-    for (int i = 0; i < NUM_TESTS; i++) {
-      srcs[i] = new Path(parent, "src" + i);
-      dsts[i] = new Path(parent, "dst" + i);
-      DFSTestUtil.createFile(fs, srcs[i], fileLen, replFactor, 1);
-      DFSTestUtil.createFile(fs, dsts[i], fileLen, replFactor, 1);
-    }
-
-    // concurrently invoking many rename
-    int start = 0, end = 0;
-    Map<Integer, Future<Void>> retFutures =
-        new HashMap<Integer, Future<Void>>();
-    for (int i = 0; i < NUM_TESTS; i++) {
-      for (;;) {
-        try {
-          LOG.info("rename #" + i);
-          Future<Void> retFuture = adfs.rename(srcs[i], dsts[i],
-              Rename.OVERWRITE);
-          retFutures.put(i, retFuture);
-          break;
-        } catch (AsyncCallLimitExceededException e) {
-          /**
-           * reached limit of async calls, fetch results of finished async calls
-           * to let follow-on calls go
-           */
-          LOG.error(e);
-          start = end;
-          end = i;
-          LOG.info(String.format("start=%d, end=%d, i=%d", start, end, i));
-          waitForReturnValues(retFutures, start, end);
-        }
-      }
-    }
-
-    // wait for completing the calls
-    waitForReturnValues(retFutures, end, NUM_TESTS);
-
-    // verify the src dir should not exist, dst should
-    verifyRenames(srcs, dsts);
-  }
-
-  private void verifyRenames(final Path[] srcs, final Path[] dsts)
-      throws IOException {
-    for (int i = 0; i < NUM_TESTS; i++) {
-      assertFalse(fs.exists(srcs[i]));
-      assertTrue(fs.exists(dsts[i]));
-    }
-  }
-
-  void waitForReturnValues(final Map<Integer, Future<Void>> retFutures,
-      final int start, final int end)
-      throws InterruptedException, ExecutionException {
-    TestAsyncDFS.waitForReturnValues(retFutures, start, end);
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7adc4d17/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestAsyncHDFSWithHA.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestAsyncHDFSWithHA.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestAsyncHDFSWithHA.java
deleted file mode 100644
index 9ade8ec..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestAsyncHDFSWithHA.java
+++ /dev/null
@@ -1,181 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hdfs;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hdfs.protocol.ClientProtocol;
-import org.apache.hadoop.hdfs.server.namenode.ha.HATestUtil;
-import org.apache.hadoop.io.retry.AsyncCallHandler;
-import org.apache.hadoop.io.retry.RetryInvocationHandler;
-import org.apache.hadoop.ipc.Client;
-import org.apache.hadoop.security.UserGroupInformation;
-import org.apache.hadoop.test.GenericTestUtils;
-import org.apache.hadoop.util.concurrent.AsyncGetFuture;
-import org.apache.log4j.Level;
-import org.junit.Assert;
-import org.junit.Test;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
-import java.util.concurrent.Future;
-
-/** Test async methods with HA setup. */
-public class TestAsyncHDFSWithHA {
-  static final Logger LOG = LoggerFactory.getLogger(TestAsyncHDFSWithHA.class);
-  static {
-    GenericTestUtils.setLogLevel(RetryInvocationHandler.LOG, Level.ALL);
-  }
-
-  private static <T> Future<T> getReturnValue() {
-    return new AsyncGetFuture<>(AsyncCallHandler.getAsyncReturn());
-  }
-
-  static void mkdirs(DistributedFileSystem dfs, String dir, Path[] srcs,
-                     Path[] dsts) throws IOException {
-    for (int i = 0; i < srcs.length; i++) {
-      srcs[i] = new Path(dir, "src" + i);
-      dsts[i] = new Path(dir, "dst" + i);
-      dfs.mkdirs(srcs[i]);
-    }
-  }
-
-  static void runTestAsyncWithoutRetry(Configuration conf,
-      MiniDFSCluster cluster, DistributedFileSystem dfs) throws Exception {
-    final int num = 5;
-
-    final String renameDir = "/testAsyncWithoutRetry/";
-    final Path[] srcs = new Path[num + 1];
-    final Path[] dsts = new Path[num + 1];
-    mkdirs(dfs, renameDir, srcs, dsts);
-
-    // create a proxy without retry.
-    final NameNodeProxiesClient.ProxyAndInfo<ClientProtocol> proxyInfo
-        = NameNodeProxies.createNonHAProxy(conf,
-        cluster.getNameNode(0).getNameNodeAddress(),
-        ClientProtocol.class, UserGroupInformation.getCurrentUser(),
-        false);
-    final ClientProtocol cp = proxyInfo.getProxy();
-
-    // submit async calls
-    Client.setAsynchronousMode(true);
-    final List<Future<Void>> results = new ArrayList<>();
-    for (int i = 0; i < num; i++) {
-      final String src = srcs[i].toString();
-      final String dst = dsts[i].toString();
-      LOG.info(i + ") rename " + src + " -> " + dst);
-      cp.rename2(src, dst);
-      results.add(getReturnValue());
-    }
-    Client.setAsynchronousMode(false);
-
-    // wait for the async calls
-    for (Future<Void> f : results) {
-      f.get();
-    }
-
-    //check results
-    for (int i = 0; i < num; i++) {
-      Assert.assertEquals(false, dfs.exists(srcs[i]));
-      Assert.assertEquals(true, dfs.exists(dsts[i]));
-    }
-  }
-
-  /** Testing HDFS async methods with HA setup. */
-  @Test(timeout = 120000)
-  public void testAsyncWithHAFailover() throws Exception {
-    final int num = 10;
-
-    final Configuration conf = new HdfsConfiguration();
-    final MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf)
-        .nnTopology(MiniDFSNNTopology.simpleHATopology())
-        .numDataNodes(0).build();
-
-    try {
-      cluster.waitActive();
-      cluster.transitionToActive(0);
-
-      final DistributedFileSystem dfs = HATestUtil.configureFailoverFs(
-          cluster, conf);
-      runTestAsyncWithoutRetry(conf, cluster, dfs);
-
-      final String renameDir = "/testAsyncWithHAFailover/";
-      final Path[] srcs = new Path[num + 1];
-      final Path[] dsts = new Path[num + 1];
-      mkdirs(dfs, renameDir, srcs, dsts);
-
-      // submit async calls and trigger failover in the middle.
-      final AsyncDistributedFileSystem adfs
-          = dfs.getAsyncDistributedFileSystem();
-      final ExecutorService executor = Executors.newFixedThreadPool(num + 1);
-
-      final List<Future<Void>> results = new ArrayList<>();
-      final List<IOException> exceptions = new ArrayList<>();
-      final List<Future<?>> futures = new ArrayList<>();
-      final int half = num/2;
-      for(int i = 0; i <= num; i++) {
-        final int id = i;
-        futures.add(executor.submit(new Runnable() {
-          @Override
-          public void run() {
-            try {
-              if (id == half) {
-                // failover
-                cluster.shutdownNameNode(0);
-                cluster.transitionToActive(1);
-              } else {
-                // rename
-                results.add(adfs.rename(srcs[id], dsts[id]));
-              }
-            } catch (IOException e) {
-              exceptions.add(e);
-            }
-          }
-        }));
-      }
-
-      // wait for the tasks
-      Assert.assertEquals(num + 1, futures.size());
-      for(int i = 0; i <= num; i++) {
-        futures.get(i).get();
-      }
-      // wait for the async calls
-      Assert.assertEquals(num, results.size());
-      Assert.assertTrue(exceptions.isEmpty());
-      for(Future<Void> r : results) {
-        r.get();
-      }
-
-      // check results
-      for(int i = 0; i <= num; i++) {
-        final boolean renamed = i != half;
-        Assert.assertEquals(!renamed, dfs.exists(srcs[i]));
-        Assert.assertEquals(renamed, dfs.exists(dsts[i]));
-      }
-    } finally {
-      if (cluster != null) {
-        cluster.shutdown();
-      }
-    }
-  }
-}
\ 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


[11/17] hadoop git commit: HDFS-10448. CacheManager#addInternal tracks bytesNeeded incorrectly when dealing with replication factors other than 1 (Yiqun Lin via cmccabe)

Posted by ae...@apache.org.
HDFS-10448. CacheManager#addInternal tracks bytesNeeded incorrectly when dealing with replication factors other than 1 (Yiqun Lin via cmccabe)


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

Branch: refs/heads/HDFS-1312
Commit: 46f1602e896273b308fbd5df6c75f6c142828227
Parents: 7adc4d1
Author: Colin Patrick Mccabe <cm...@cloudera.com>
Authored: Mon Jun 20 18:25:09 2016 -0700
Committer: Colin Patrick Mccabe <cm...@cloudera.com>
Committed: Mon Jun 20 18:25:09 2016 -0700

----------------------------------------------------------------------
 .../org/apache/hadoop/hdfs/server/namenode/CacheManager.java    | 5 ++---
 1 file changed, 2 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/46f1602e/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/CacheManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/CacheManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/CacheManager.java
index cd57c56..366dd9b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/CacheManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/CacheManager.java
@@ -396,8 +396,7 @@ public final class CacheManager {
     if (pool.getLimit() == CachePoolInfo.LIMIT_UNLIMITED) {
       return;
     }
-    if (pool.getBytesNeeded() + (stats.getBytesNeeded() * replication) > pool
-        .getLimit()) {
+    if (pool.getBytesNeeded() + stats.getBytesNeeded() > pool.getLimit()) {
       throw new InvalidRequestException("Caching path " + path + " of size "
           + stats.getBytesNeeded() / replication + " bytes at replication "
           + replication + " would exceed pool " + pool.getPoolName()
@@ -441,7 +440,7 @@ public final class CacheManager {
       }
     }
     return new CacheDirectiveStats.Builder()
-        .setBytesNeeded(requestedBytes)
+        .setBytesNeeded(requestedBytes * replication)
         .setFilesCached(requestedFiles)
         .build();
   }


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


[09/17] hadoop git commit: HADOOP-13291. Probing stats in DFSOpsCountStatistics/S3AStorageStatistics should be correctly implemented. Contributed by Mingliang Liu.

Posted by ae...@apache.org.
HADOOP-13291. Probing stats in DFSOpsCountStatistics/S3AStorageStatistics should be correctly implemented. Contributed by Mingliang Liu.


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

Branch: refs/heads/HDFS-1312
Commit: b7c4cf7129768c0312b186dfb94ba1beb891e2f3
Parents: 5d58858
Author: Jitendra Pandey <ji...@apache.org>
Authored: Mon Jun 20 16:00:13 2016 -0700
Committer: Jitendra Pandey <ji...@apache.org>
Committed: Mon Jun 20 16:25:30 2016 -0700

----------------------------------------------------------------------
 .../hadoop/hdfs/DFSOpsCountStatistics.java      |   5 +-
 .../hadoop/hdfs/TestDFSOpsCountStatistics.java  | 108 +++++++++++++++++++
 .../hadoop/fs/s3a/S3AStorageStatistics.java     |   5 +-
 3 files changed, 114 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/b7c4cf71/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSOpsCountStatistics.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSOpsCountStatistics.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSOpsCountStatistics.java
index d58a59f..a047d34 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSOpsCountStatistics.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSOpsCountStatistics.java
@@ -139,7 +139,8 @@ public class DFSOpsCountStatistics extends StorageStatistics {
         throw new NoSuchElementException();
       }
       final Entry<OpType, AtomicLong> entry = iterator.next();
-      return new LongStatistic(entry.getKey().name(), entry.getValue().get());
+      return new LongStatistic(entry.getKey().getSymbol(),
+          entry.getValue().get());
     }
 
     @Override
@@ -161,7 +162,7 @@ public class DFSOpsCountStatistics extends StorageStatistics {
 
   @Override
   public boolean isTracked(String key) {
-    return OpType.fromSymbol(key) == null;
+    return OpType.fromSymbol(key) != null;
   }
 
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b7c4cf71/hadoop-hdfs-project/hadoop-hdfs-client/src/test/java/org/apache/hadoop/hdfs/TestDFSOpsCountStatistics.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/test/java/org/apache/hadoop/hdfs/TestDFSOpsCountStatistics.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/test/java/org/apache/hadoop/hdfs/TestDFSOpsCountStatistics.java
new file mode 100644
index 0000000..7c1b018
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/test/java/org/apache/hadoop/hdfs/TestDFSOpsCountStatistics.java
@@ -0,0 +1,108 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hdfs;
+
+import org.apache.commons.lang3.RandomUtils;
+import org.apache.hadoop.fs.StorageStatistics.LongStatistic;
+
+import org.apache.hadoop.hdfs.DFSOpsCountStatistics.OpType;
+
+import org.junit.BeforeClass;
+import org.junit.Rule;
+import org.junit.Test;
+
+import org.junit.rules.ExpectedException;
+import org.junit.rules.Timeout;
+
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.Map;
+
+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;
+
+/**
+ * This tests basic operations of {@link DFSOpsCountStatistics} class.
+ */
+public class TestDFSOpsCountStatistics {
+
+  private static final DFSOpsCountStatistics STORAGE_STATISTICS =
+      new DFSOpsCountStatistics();
+  private static final Map<String, Long> OP_COUNTER_MAP = new HashMap<>();
+  private static final String NO_SUCH_OP = "no-such-dfs-operation-dude";
+
+  @Rule
+  public final Timeout globalTimeout = new Timeout(10 * 1000);
+  @Rule
+  public final ExpectedException exception = ExpectedException.none();
+
+  @BeforeClass
+  public static void setup() {
+    for (OpType opType : OpType.values()) {
+      final Long opCount = RandomUtils.nextLong(0, 100);
+      OP_COUNTER_MAP.put(opType.getSymbol(), opCount);
+      for (long i = 0; i < opCount; i++) {
+        STORAGE_STATISTICS.incrementOpCounter(opType);
+      }
+    }
+  }
+
+  @Test
+  public void testGetLongStatistics() {
+    short iterations = 0; // number of the iter.hasNext()
+    final Iterator<LongStatistic> iter = STORAGE_STATISTICS.getLongStatistics();
+
+    while (iter.hasNext()) {
+      final LongStatistic longStat = iter.next();
+      assertNotNull(longStat);
+      assertTrue(OP_COUNTER_MAP.containsKey(longStat.getName()));
+      assertEquals(OP_COUNTER_MAP.get(longStat.getName()).longValue(),
+          longStat.getValue());
+      iterations++;
+    }
+
+    // check that all the OpType enum entries are iterated via iter
+    assertEquals(OpType.values().length, iterations);
+  }
+
+  @Test
+  public void testGetLong() {
+    assertNull(STORAGE_STATISTICS.getLong(NO_SUCH_OP));
+
+    for (OpType opType : OpType.values()) {
+      final String key = opType.getSymbol();
+      assertEquals(OP_COUNTER_MAP.get(key), STORAGE_STATISTICS.getLong(key));
+    }
+  }
+
+  @Test
+  public void testIsTracked() {
+    assertFalse(STORAGE_STATISTICS.isTracked(NO_SUCH_OP));
+
+    final Iterator<LongStatistic> iter = STORAGE_STATISTICS.getLongStatistics();
+    while (iter.hasNext()) {
+      final LongStatistic longStatistic = iter.next();
+      assertTrue(STORAGE_STATISTICS.isTracked(longStatistic.getName()));
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b7c4cf71/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AStorageStatistics.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AStorageStatistics.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AStorageStatistics.java
index f69159a..a74b864 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AStorageStatistics.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AStorageStatistics.java
@@ -76,7 +76,8 @@ public class S3AStorageStatistics extends StorageStatistics {
         throw new NoSuchElementException();
       }
       final Map.Entry<Statistic, AtomicLong> entry = iterator.next();
-      return new LongStatistic(entry.getKey().name(), entry.getValue().get());
+      return new LongStatistic(entry.getKey().getSymbol(),
+          entry.getValue().get());
     }
 
     @Override
@@ -98,7 +99,7 @@ public class S3AStorageStatistics extends StorageStatistics {
 
   @Override
   public boolean isTracked(String key) {
-    return Statistic.fromSymbol(key) == null;
+    return Statistic.fromSymbol(key) != null;
   }
 
 }


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


[03/17] hadoop git commit: HADOOP-13280. FileSystemStorageStatistics#getLong(“readOps“) should return readOps + largeReadOps (Mingliang Liu via cmccabe)

Posted by ae...@apache.org.
HADOOP-13280. FileSystemStorageStatistics#getLong(\u201creadOps\u201c) should return readOps + largeReadOps (Mingliang Liu via cmccabe)


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

Branch: refs/heads/HDFS-1312
Commit: 5370a6ffaec5227c0978f10c86a5811155271933
Parents: 5f6b415
Author: Colin Patrick Mccabe <cm...@cloudera.com>
Authored: Mon Jun 20 10:43:07 2016 -0700
Committer: Colin Patrick Mccabe <cm...@cloudera.com>
Committed: Mon Jun 20 10:46:18 2016 -0700

----------------------------------------------------------------------
 .../hadoop/fs/FileSystemStorageStatistics.java  |   2 +-
 .../fs/TestFileSystemStorageStatistics.java     | 134 +++++++++++++++++++
 2 files changed, 135 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/5370a6ff/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileSystemStorageStatistics.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileSystemStorageStatistics.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileSystemStorageStatistics.java
index 14f7cdd..f179cce 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileSystemStorageStatistics.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileSystemStorageStatistics.java
@@ -87,7 +87,7 @@ public class FileSystemStorageStatistics extends StorageStatistics {
     case "bytesWritten":
       return data.getBytesWritten();
     case "readOps":
-      return Long.valueOf(data.getReadOps());
+      return (long) (data.getReadOps() + data.getLargeReadOps());
     case "largeReadOps":
       return Long.valueOf(data.getLargeReadOps());
     case "writeOps":

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5370a6ff/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestFileSystemStorageStatistics.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestFileSystemStorageStatistics.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestFileSystemStorageStatistics.java
new file mode 100644
index 0000000..59c3b8d
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestFileSystemStorageStatistics.java
@@ -0,0 +1,134 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs;
+
+import org.apache.commons.lang.math.RandomUtils;
+import org.apache.hadoop.fs.StorageStatistics.LongStatistic;
+
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+
+import org.junit.rules.ExpectedException;
+import org.junit.rules.Timeout;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Iterator;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+
+/**
+ * This tests basic operations of {@link FileSystemStorageStatistics} class.
+ */
+public class TestFileSystemStorageStatistics {
+  private static final Logger LOG = LoggerFactory.getLogger(
+      TestFileSystemStorageStatistics.class);
+  private static final String FS_STORAGE_STATISTICS_NAME = "test-fs-statistics";
+  private static final String[] STATISTICS_KEYS = {
+      "bytesRead",
+      "bytesWritten",
+      "readOps",
+      "largeReadOps",
+      "writeOps",
+      "bytesReadLocalHost",
+      "bytesReadDistanceOfOneOrTwo",
+      "bytesReadDistanceOfThreeOrFour",
+      "bytesReadDistanceOfFiveOrLarger"
+  };
+
+  private FileSystem.Statistics statistics =
+      new FileSystem.Statistics("test-scheme");
+  private FileSystemStorageStatistics storageStatistics =
+      new FileSystemStorageStatistics(FS_STORAGE_STATISTICS_NAME, statistics);
+
+  @Rule
+  public final Timeout globalTimeout = new Timeout(10 * 1000);
+  @Rule
+  public final ExpectedException exception = ExpectedException.none();
+
+  @Before
+  public void setup() {
+    statistics.incrementBytesRead(RandomUtils.nextInt(100));
+    statistics.incrementBytesWritten(RandomUtils.nextInt(100));
+    statistics.incrementLargeReadOps(RandomUtils.nextInt(100));
+    statistics.incrementWriteOps(RandomUtils.nextInt(100));
+
+    statistics.incrementBytesReadByDistance(0, RandomUtils.nextInt(100));
+    statistics.incrementBytesReadByDistance(1, RandomUtils.nextInt(100));
+    statistics.incrementBytesReadByDistance(3, RandomUtils.nextInt(100));
+  }
+
+  @Test
+  public void testgetLongStatistics() {
+    Iterator<LongStatistic> iter = storageStatistics.getLongStatistics();
+    while (iter.hasNext()) {
+      final LongStatistic longStat = iter.next();
+      assertNotNull(longStat);
+      final long expectedStat = getStatisticsValue(longStat.getName());
+      LOG.info("{}: FileSystem.Statistics={}, FileSystemStorageStatistics={}",
+          longStat.getName(), expectedStat, longStat.getValue());
+      assertEquals(expectedStat, longStat.getValue());
+    }
+  }
+
+  @Test
+  public void testGetLong() {
+    for (String key : STATISTICS_KEYS) {
+      final long expectedStat = getStatisticsValue(key);
+      final long storageStat = storageStatistics.getLong(key);
+      LOG.info("{}: FileSystem.Statistics={}, FileSystemStorageStatistics={}",
+          key, expectedStat, storageStat);
+      assertEquals(expectedStat, storageStat);
+    }
+  }
+
+  /**
+   * Helper method to retrieve the specific FileSystem.Statistics value by name.
+   *
+   * Basically, the {@link FileSystemStorageStatistics} should do this
+   * internally in a similar approach.
+   */
+  private long getStatisticsValue(String name) {
+    switch (name) {
+    case "bytesRead":
+      return statistics.getBytesRead();
+    case "bytesWritten":
+      return statistics.getBytesWritten();
+    case "readOps":
+      return statistics.getReadOps();
+    case "largeReadOps":
+      return statistics.getLargeReadOps();
+    case "writeOps":
+      return statistics.getWriteOps();
+    case "bytesReadLocalHost":
+      return statistics.getBytesReadByDistance(0);
+    case "bytesReadDistanceOfOneOrTwo":
+      return statistics.getBytesReadByDistance(1);
+    case "bytesReadDistanceOfThreeOrFour":
+      return statistics.getBytesReadByDistance(3);
+    case "bytesReadDistanceOfFiveOrLarger":
+      return statistics.getBytesReadByDistance(5);
+    default:
+      return 0;
+    }
+  }
+
+}


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